diff --git a/.gitignore b/.gitignore index 160957d13..9de2a521b 100644 --- a/.gitignore +++ b/.gitignore @@ -6,3 +6,9 @@ /v15 npm-debug.log /DataProcessing/DataX.Utilities/DataX.Utility.CodeSign/obj +/Tests/ScenarioTester/ScenarioTester/bin +/Tests/ScenarioTester/.vs +*.suo +/Tests/ScenarioTester/ScenarioTester/obj +/Tests/ScenarioTester/ScenarioTesterTests/bin +/Tests/ScenarioTester/ScenarioTesterTests/obj diff --git a/DataProcessing/DataX.Utilities/.gitignore b/DataProcessing/DataX.Utilities/.gitignore new file mode 100644 index 000000000..28a111476 --- /dev/null +++ b/DataProcessing/DataX.Utilities/.gitignore @@ -0,0 +1 @@ +*/obj \ No newline at end of file diff --git a/DataProcessing/DataX.Utilities/DataX.Utility.CodeSign/DataX.Utility.CodeSign.csproj b/DataProcessing/DataX.Utilities/DataX.Utility.CodeSign/DataX.Utility.CodeSign.csproj index e57de3f80..98941e0f3 100644 --- a/DataProcessing/DataX.Utilities/DataX.Utility.CodeSign/DataX.Utility.CodeSign.csproj +++ b/DataProcessing/DataX.Utilities/DataX.Utility.CodeSign/DataX.Utility.CodeSign.csproj @@ -2,7 +2,7 @@ Library - netcoreapp2.1 + netcoreapp2.2 true diff --git a/DataProcessing/Spark.nuspec b/DataProcessing/Spark.nuspec index 6173f76bb..81cec908f 100644 --- a/DataProcessing/Spark.nuspec +++ b/DataProcessing/Spark.nuspec @@ -20,16 +20,21 @@ - - - - - + + + + + + + + + + - + \ No newline at end of file diff --git a/DataProcessing/datax-core/pom.xml b/DataProcessing/datax-core/pom.xml index 3c4dcd4e7..2fa0a0373 100644 --- a/DataProcessing/datax-core/pom.xml +++ b/DataProcessing/datax-core/pom.xml @@ -50,15 +50,15 @@ SOFTWARE com.microsoft.datax - datax-core_2.3_2.11 - 1.1.0 + datax-core_2.4_2.11 + 1.2.0 Data Accelerator Core This package contains the core module of Data Accelerator functionality. https://github.com/Microsoft/data-accelerator jar - 2.3.0 + 2.4.0 2.11 8 ${scala.version.major}.${scala.version.minor} diff --git a/DataProcessing/datax-core/src/main/scala/datax/constants/BlobProperties.scala b/DataProcessing/datax-core/src/main/scala/datax/constants/BlobProperties.scala new file mode 100644 index 000000000..6fe44c805 --- /dev/null +++ b/DataProcessing/datax-core/src/main/scala/datax/constants/BlobProperties.scala @@ -0,0 +1,10 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +package datax.constants + +object BlobProperties { + // Define constants for blobs + val BlobHostPath = ".blob.core.windows.net" +} diff --git a/DataProcessing/datax-core/src/main/scala/datax/constants/JobArgument.scala b/DataProcessing/datax-core/src/main/scala/datax/constants/JobArgument.scala index d2ff33903..c1e38e758 100644 --- a/DataProcessing/datax-core/src/main/scala/datax/constants/JobArgument.scala +++ b/DataProcessing/datax-core/src/main/scala/datax/constants/JobArgument.scala @@ -15,4 +15,7 @@ object JobArgument { def ConfName_AppInsightKeyRef = s"${ConfNamePrefix}APPINSIGHTKEYREF" def ConfName_BlobWriterTimeout: String = s"${ConfNamePrefix}BlobWriterTimeout" def ConfName_DefaultVaultName: String = s"${ConfNamePrefix}DEFAULTVAULTNAME" + def ConfName_DefaultStorageAccount: String = s"${ConfNamePrefix}DEFAULTSTORAGEACCOUNT" + def ConfName_DefaultContainer: String = s"${ConfNamePrefix}DEFAULTCONTAINER" + def ConfName_AzureStorageJarPath: String = s"${ConfNamePrefix}AZURESTORAGEJARPATH" } diff --git a/DataProcessing/datax-core/src/main/scala/datax/sink/package.scala b/DataProcessing/datax-core/src/main/scala/datax/sink/package.scala index e8a13a9f1..16033f7f4 100644 --- a/DataProcessing/datax-core/src/main/scala/datax/sink/package.scala +++ b/DataProcessing/datax-core/src/main/scala/datax/sink/package.scala @@ -10,7 +10,8 @@ import datax.config.SettingDictionary import org.apache.spark.sql.{DataFrame, Row, SparkSession} package object sink { - type SinkDelegate = (Row, Seq[Row], Timestamp, Int, String)=>Map[String, Int] + type JsonSinkDelegate = (Row, Seq[Row], Timestamp, Int, String)=>Map[String, Int] + type SinkDelegate = (DataFrame, Timestamp, String)=>Map[String, Int] type Metrics = Map[String, Double] trait SinkOperatorFactory{ @@ -21,6 +22,7 @@ package object sink { case class SinkOperator(name: String, isEnabled: Boolean, + sinkAsJson: Boolean, flagColumnExprGenerator: () => String, generator: (Int)=>SinkDelegate, onInitialization: (SparkSession)=>Unit = null, diff --git a/DataProcessing/datax-host/pom.xml b/DataProcessing/datax-host/pom.xml index 6dbfed3bb..1c1fa0e76 100644 --- a/DataProcessing/datax-host/pom.xml +++ b/DataProcessing/datax-host/pom.xml @@ -50,11 +50,11 @@ SOFTWARE com.microsoft.datax - datax-host_2.3_2.11 - 1.1.0 + datax-host_2.4_2.11 + 1.2.0 - 2.3.0 + 2.4.0 2.11 8 ${scala.version.major}.${scala.version.minor} @@ -117,13 +117,13 @@ SOFTWARE com.microsoft.datax - datax-core_2.3_2.11 - 1.1.0 + datax-core_2.4_2.11 + 1.2.0 com.microsoft.datax - datax-utility_2.3_2.11 - 1.1.0 + datax-utility_2.4_2.11 + 1.2.0 com.microsoft.azure @@ -138,21 +138,44 @@ SOFTWARE com.microsoft.azure azure-storage - 5.3.0 + 3.1.0 com.microsoft.azure azure-documentdb 1.16.1 + + org.apache.hadoop + hadoop-azure + 2.7.3 + + + org.mortbay.jetty + jetty-util + 6.1.25 + + + org.json + json + 20180813 + org.apache.kafka kafka-clients 2.0.0 + + com.microsoft.azure + azure-sqldb-spark + 1.0.2 + + + com.databricks + dbutils-api_2.11 + 0.0.3 + - - build @@ -228,24 +251,6 @@ SOFTWARE - - maven-assembly-plugin - 2.4.1 - - - with-dependencies.xml - - - - - make-assembly - package - - single - - - - com.github.github site-maven-plugin diff --git a/DataProcessing/datax-host/src/main/scala/datax/app/BatchApp.scala b/DataProcessing/datax-host/src/main/scala/datax/app/BatchApp.scala index e510b7a45..47ea2b223 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/app/BatchApp.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/app/BatchApp.scala @@ -11,6 +11,6 @@ object BatchApp { def main(inputArguments: Array[String]): Unit = { BlobBatchingHost.runBatchApp( inputArguments, - config => CommonProcessorFactory.createProcessor(config).asBlobPointerProcessor()) + config => CommonProcessorFactory.createProcessor(config).asBatchBlobProcessor()) } } diff --git a/DataProcessing/datax-host/src/main/scala/datax/client/sql/SqlConf.scala b/DataProcessing/datax-host/src/main/scala/datax/client/sql/SqlConf.scala new file mode 100644 index 000000000..ce9cf6372 --- /dev/null +++ b/DataProcessing/datax-host/src/main/scala/datax/client/sql/SqlConf.scala @@ -0,0 +1,26 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +package datax.client.sql + +case class SqlConf(name: String, + connectionString: String, + url: String, + encrypt: String, + trustServerCertificate: String, + hostNameInCertificate: String, + databaseName:String, + table: String, + writeMode: String, + userName:String, + password:String, + filter: String, + connectionTimeout: String, + queryTimeout: String, + useBulkCopy : Boolean, + useBulkCopyTableLock: String, + useBulkCopyInternalTransaction: String, + bulkCopyTimeout:String, + bulkCopyBatchSize:String + ) diff --git a/DataProcessing/datax-host/src/main/scala/datax/executor/ExecutorHelper.scala b/DataProcessing/datax-host/src/main/scala/datax/executor/ExecutorHelper.scala new file mode 100644 index 000000000..863069d0a --- /dev/null +++ b/DataProcessing/datax-host/src/main/scala/datax/executor/ExecutorHelper.scala @@ -0,0 +1,85 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +package datax.executor + +import datax.config.ConfigManager +import datax.constants.{BlobProperties, JobArgument} +import datax.fs.HadoopClient +import datax.host.SparkJarLoader +import datax.securedsetting.KeyVaultClient +import org.apache.log4j.LogManager +import org.apache.spark.broadcast +import org.apache.spark.sql.SparkSession + +object ExecutorHelper { + private val logger = LogManager.getLogger(this.getClass) + + /*** + * Create broadcast variable for blob storage account key + * @param path blob storage path + * @param spark SparkSession + */ + def createBlobStorageKeyBroadcastVariable(path: String, spark : SparkSession): broadcast.Broadcast[String] ={ + addJarToExecutor(spark) + val sc = spark.sparkContext + val sa = getStorageAccountName(path) + var key = "" + KeyVaultClient.withKeyVault {vaultName => key = HadoopClient.resolveStorageAccount(vaultName, sa).get} + val blobStorageKey = sc.broadcast(key) + blobStorageKey + } + + /*** + * Get the storage account name from blob path + * @param path blob storage path + */ + private def getStorageAccountName(path:String):String ={ + val regex = s"@([a-zA-Z0-9-_]+)${BlobProperties.BlobHostPath}".r + regex.findFirstMatchIn(path) match { + case Some(partition) => partition.group(1) + case None => null + } + } + + /*** + * Add azure-storage jar to executor nodes + * @param spark Spark Session + */ + private def addJarToExecutor(spark : SparkSession){ + try{ + logger.warn("Adding azure-storage jar to executor nodes") + withStorageAccount {(storageAccount,containerName,azureStorageJarPath) => SparkJarLoader.addJar(spark, s"wasbs://$containerName@$storageAccount${BlobProperties.BlobHostPath}$azureStorageJarPath")} + } + catch { + case e: Exception => { + logger.error(s"azure-storage jar could not be added to executer nodes", e) + throw e + } + } + } + + /*** + * a scope to execute operation with the default storageAccount/container/azureStorageJarPath, skip the operation if that doesn't exist. + * @param callback execution within the scope + */ + private def withStorageAccount(callback: (String, String, String)=> Unit) = { + ConfigManager.getActiveDictionary().get(JobArgument.ConfName_DefaultStorageAccount) match { + case Some(storageAccount) => + logger.warn(s"Default Storage Account is $storageAccount") + ConfigManager.getActiveDictionary().get(JobArgument.ConfName_DefaultContainer) match { + case Some(containerName) => + logger.warn(s"Default container is $containerName") + ConfigManager.getActiveDictionary().get(JobArgument.ConfName_AzureStorageJarPath) match { + case Some(azureStorageJarPath) => + logger.warn(s"Azure storage jar path is $azureStorageJarPath") + callback(storageAccount, containerName, azureStorageJarPath) + case None => logger.warn(s"No azure storage jar path is defined") + } + case None => logger.warn(s"No default container is defined") + } + case None => logger.warn(s"No default storage account is defined") + } + } +} diff --git a/DataProcessing/datax-host/src/main/scala/datax/fs/HadoopClient.scala b/DataProcessing/datax-host/src/main/scala/datax/fs/HadoopClient.scala index 830fc53ea..1b93c3cbf 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/fs/HadoopClient.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/fs/HadoopClient.scala @@ -13,7 +13,7 @@ import java.util.zip.GZIPInputStream import com.google.common.io.{Files => GFiles} import datax.config.SparkEnvVariables -import datax.constants.ProductConstant +import datax.constants.{ProductConstant, BlobProperties} import datax.exception.EngineException import datax.securedsetting.KeyVaultClient import datax.telemetry.AppInsightLogger @@ -21,6 +21,7 @@ import org.apache.commons.codec.digest.DigestUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path, RemoteIterator} import org.apache.log4j.LogManager +import org.apache.spark.broadcast import scala.language.implicitConversions import scala.collection.mutable @@ -41,8 +42,11 @@ object HadoopClient { * @param conf hadoop configuration for initialization */ def setConf(conf: Configuration = null): Unit ={ - if(conf==null) + if(conf==null) { hadoopConf = new Configuration() + //Used to fetch fileSystem for wasbs + hadoopConf.set("fs.wasbs.impl","org.apache.hadoop.fs.azure.NativeAzureFileSystem") + } else hadoopConf = conf } @@ -71,7 +75,7 @@ object HadoopClient { val scheme = uri.getScheme if(scheme == "wasb" || scheme == "wasbs") Option(uri.getHost) match { - case Some(host) => host.toLowerCase().replace(".blob.core.windows.net", "") + case Some(host) => host.toLowerCase().replace(s"${BlobProperties.BlobHostPath}", "") case None => null } else @@ -99,17 +103,17 @@ object HadoopClient { */ private def setStorageAccountKey(sa: String, key: String): Unit ={ storageAccountKeys.synchronized{ - storageAccountKeys += sa->key + storageAccountKeys += sa->key } // get the default storage account val defaultFS = getConf().get("fs.defaultFS","") // set the key only if its a non-default storage account - if(!defaultFS.toLowerCase().contains(s"$sa.blob.core.windows.net")) { + if(!defaultFS.toLowerCase().contains(s"$sa${BlobProperties.BlobHostPath}")) { logger.warn(s"Setting the key in hdfs conf for storage account $sa") - getConf().set(s"fs.azure.account.key.$sa.blob.core.windows.net", key) + setStorageAccountKeyOnHadoopConf(sa, key) } - else { + else { logger.warn(s"Default storage account $sa found, skipping setting the key") } } @@ -120,27 +124,46 @@ object HadoopClient { * warn if key is not found but we let it continue so static key settings outside of the job can still work * @param vaultName key vault name to get the key of storage account * @param sa name of the storage account + * @param blobStorageKey broadcasted storage account key */ - private def resolveStorageAccount(vaultName: String, sa: String) = { - val secretId = s"keyvault://$vaultName/${ProductConstant.ProductRoot}-sa-$sa" - KeyVaultClient.getSecret(secretId) match { - case Some(value)=> - logger.warn(s"Retrieved key for storage account '$sa' with secretid:'$secretId'") - setStorageAccountKey(sa, value) - case None => - logger.warn(s"Failed to find key for storage account '$sa' with secretid:'$secretId'") + def resolveStorageAccount(vaultName: String, sa: String, blobStorageKey: broadcast.Broadcast[String] = null) : Option[String] = { + if(blobStorageKey != null) { + setStorageAccountKey(sa, blobStorageKey.value) + Some(blobStorageKey.value) + } + else { + // Fetch secret from keyvault using KeyVaultMsiAuthenticatorClient and if that does not return secret then fetch it using secret scope + val secretId = s"keyvault://$vaultName/${ProductConstant.ProductRoot}-sa-$sa" + KeyVaultClient.getSecret(secretId) match { + case Some(value)=> + logger.warn(s"Retrieved key for storage account '$sa' with secretid:'$secretId'") + setStorageAccountKey(sa, value) + Some(value) + case None => + val databricksSecretId = s"secretscope://$vaultName/${ProductConstant.ProductRoot}-sa-$sa" + KeyVaultClient.getSecret(databricksSecretId) match { + case Some(value)=> + logger.warn(s"Retrieved key for storage account '$sa' with secretid:'$databricksSecretId'") + setStorageAccountKey(sa, value) + Some(value) + case None => + logger.warn(s"Failed to find key for storage account '$sa' with secretid:'$secretId' and '$databricksSecretId'") + None + } + } } } /*** * set key for storage account required by the specified hdfs path * @param path hdfs file to resolve the key of storage account if it is a valid wasb/wasbs path, do nothing if it isn't + * @param blobStorageKey broadcasted storage account key */ - private def resolveStorageAccountKeyForPath(path: String) = { + private def resolveStorageAccountKeyForPath(path: String, blobStorageKey: broadcast.Broadcast[String] = null) = { val sa = getWasbStorageAccount(path) if(sa != null && !sa.isEmpty){ - KeyVaultClient.withKeyVault {vaultName => resolveStorageAccount(vaultName, sa)} + KeyVaultClient.withKeyVault {vaultName => resolveStorageAccount(vaultName, sa, blobStorageKey)} } } @@ -214,15 +237,16 @@ object HadoopClient { * read a hdfs file * @param hdfsPath path to the hdfs file * @param gzip whether it is a gzipped file + * @param blobStorageKey storage account key broadcast variable * @throws IOException if any * @return a iterable of strings from content of the file */ @throws[IOException] - def readHdfsFile(hdfsPath: String, gzip:Boolean=false): Iterable[String] = { + def readHdfsFile(hdfsPath: String, gzip:Boolean=false, blobStorageKey: broadcast.Broadcast[String] = null): Iterable[String] = { val logger = LogManager.getLogger(s"FileLoader${SparkEnvVariables.getLoggerSuffix()}") // resolve key to access azure storage account - resolveStorageAccountKeyForPath(hdfsPath) + resolveStorageAccountKeyForPath(hdfsPath, blobStorageKey) val lines = new ListBuffer[String] val t1= System.nanoTime() @@ -304,15 +328,17 @@ object HadoopClient { * @param content conent to write into the file * @param timeout timeout duration for the write operation, by default 5 seconds * @param retries times in retries, by default 0 meaning no retries. + * @param blobStorageKey storage account key broadcast variable */ def writeWithTimeoutAndRetries(hdfsPath: String, content: Array[Byte], timeout: Duration = Duration(5, TimeUnit.SECONDS), - retries: Int = 0 + retries: Int = 0, + blobStorageKey: broadcast.Broadcast[String] ) = { val logger = LogManager.getLogger(s"FileWriter${SparkEnvVariables.getLoggerSuffix()}") def f = Future{ - writeHdfsFile(hdfsPath, content, getConf(), false) + writeHdfsFile(hdfsPath, content, getConf(), false, blobStorageKey) } var remainingAttempts = retries+1 while(remainingAttempts>0) { @@ -330,6 +356,15 @@ object HadoopClient { } } + /** + * set storage account key on hadoop conf + * @param sa storage account name + * @param value storage account key + */ + private def setStorageAccountKeyOnHadoopConf(sa: String, value: String): Unit = { + getConf().set(s"fs.azure.account.key.$sa${BlobProperties.BlobHostPath}", value) + } + /** * make sure parent folder exists for path, create the folder if it doesn't exist * @param path specified path to check its parent folder @@ -349,11 +384,12 @@ object HadoopClient { * @param content content to write into the file * @param conf hadoop configuration * @param overwriteIfExists flag to specify if the file needs to be overwritten if it already exists in hdfs + * @param blobStorageKey storage account key broadcast variable * @throws IOException if any from lower file system operation */ @throws[IOException] - def writeHdfsFile(hdfsPath: String, content: Array[Byte], conf: Configuration, overwriteIfExists:Boolean) { - resolveStorageAccountKeyForPath(hdfsPath) + private def writeHdfsFile(hdfsPath: String, content: Array[Byte], conf: Configuration, overwriteIfExists:Boolean, blobStorageKey: broadcast.Broadcast[String] = null) { + resolveStorageAccountKeyForPath(hdfsPath, blobStorageKey) val logger = LogManager.getLogger("writeHdfsFile") @@ -432,6 +468,7 @@ object HadoopClient { * @return a list of file paths under the folder */ def listFiles(folder: String): Iterator[String] = { + resolveStorageAccountKeyForPath(folder) val path = new Path(folder) val fs = path.getFileSystem(getConf) diff --git a/DataProcessing/datax-host/src/main/scala/datax/handler/ExtendedUDFHandler.scala b/DataProcessing/datax-host/src/main/scala/datax/handler/ExtendedUDFHandler.scala index 7ebd3f5be..f15093a31 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/handler/ExtendedUDFHandler.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/handler/ExtendedUDFHandler.scala @@ -51,9 +51,11 @@ object ExtendedUDFHandler { val returnType = ScalaReflection.schemaFor(typeArgs.last).dataType val udf = clazz.newInstance() val argumentCount = typeArgs.length - 1 - + val inputsNullSafe = typeArgs.take(argumentCount).map(t=>{ + false + }) val wrap = generateFunctionRef(udf, argumentCount, spark, dict) - registerFunction(spark, name, wrap.func, returnType, argumentCount) + registerFunction(spark, name, wrap.func, returnType, argumentCount, inputsNullSafe) wrap.onInterval } } @@ -90,9 +92,9 @@ object ExtendedUDFHandler { UdfWrap(obj.func.apply(_:Any, _:Any, _:Any), obj.onInterval) } - def registerFunction(spark:SparkSession, name: String, func: AnyRef, returnType: DataType, argumentCount: Int) = { + def registerFunction(spark:SparkSession, name: String, func: AnyRef, returnType: DataType, argumentCount: Int, inputsNullSafe: Seq[Boolean]) = { def builder(e: Seq[Expression]) = if (e.length == argumentCount) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, inputsNullSafe=inputsNullSafe, udfName = Some(name)) } else { throw new EngineException(s"Invalid number of arguments for function $name. Expected: $argumentCount; Found: ${e.length}") } diff --git a/DataProcessing/datax-host/src/main/scala/datax/host/BlobBatchingHost.scala b/DataProcessing/datax-host/src/main/scala/datax/host/BlobBatchingHost.scala index 0d6440bab..5aa7ac3d1 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/host/BlobBatchingHost.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/host/BlobBatchingHost.scala @@ -7,47 +7,65 @@ package datax.host import java.sql.Timestamp import java.text.SimpleDateFormat import java.time.Instant -import java.util.concurrent.Executors +import java.time.temporal.ChronoUnit import datax.config.UnifiedConfig import datax.constants.ProductConstant +import datax.data.FileInternal import datax.fs.HadoopClient import datax.input.BatchBlobInputSetting -import datax.processor.BlobPointerProcessor +import datax.processor.{BatchBlobProcessor, CommonProcessorFactory} import datax.telemetry.AppInsightLogger -import datax.utility.DataMerger import org.apache.log4j.LogManager import scala.language.postfixOps import scala.collection.mutable.{HashSet, ListBuffer} -import scala.collection.parallel.ExecutionContextTaskSupport -import scala.concurrent.ExecutionContext import scala.concurrent.duration._ object BlobBatchingHost { - def getInputBlobPathPrefixes(prefix: String, datetimeFormat: String, startTime: Instant, durationInSeconds: Long, timeIntervalInSeconds: Long):Iterable[(String, Timestamp)] = { + val appLog = LogManager.getLogger("runBatchApp") + + def getInputBlobPathPrefixes(path: String, startTime: Instant, processingWindowInSeconds: Long, partitionIncrementDurationInSeconds: Long):Iterable[(String, Timestamp)] = { val result = new ListBuffer[(String, Timestamp)] val cache = new HashSet[String] + val datetimeFormat = getDateTimePattern(path) var t:Long = 0 - //val utcZoneId = ZoneId.of("UTC") - val dateFormat = new SimpleDateFormat(datetimeFormat) - while(tBlobPointerProcessor ) = { - val appLog = LogManager.getLogger("runBatchApp") + // Get the datetime pattern like {yyyy-MM-dd} in the input path. + // For e.g. if the input path is "wasbs://outputs@myaccount.blob.core.windows.net/{yyyy-MM-dd}/flow1", this will return yyyy-MM-dd + private def getDateTimePattern(inputPath:String):String ={ + val regex = """\{([yMdHmsS\-/.]+)\}*""".r + + regex.findFirstMatchIn(inputPath) match { + case Some(partition) => partition.group(1) + case None => + appLog.warn(s"InputPath string does not contain the datetime pattern ${regex.regex}.") + "" + } + } + + def runBatchApp(inputArguments: Array[String],processorGenerator: UnifiedConfig=>BatchBlobProcessor ) = { val (appHost, config) = CommonAppHost.initApp(inputArguments) appLog.warn(s"Batch Mode Work Started") @@ -56,48 +74,33 @@ object BlobBatchingHost { AppInsightLogger.trackEvent(ProductConstant.ProductRoot + "/batch/app/begin") val prefixes = blobsConf.flatMap(blobs=>{ - val inputBlobPathPrefix = blobs.pathPrefix - val inputBlobDateTimeFormat = blobs.pathPartitionFolderFormat + val inputBlobPath= blobs.path val inputBlobStartTime = Instant.parse(blobs.startTime) - val inputBlobDurationInHours = blobs.durationInHours - val inputBlobTimeIntervalInHours = 1 + val inputBlobEndTime = Instant.parse(blobs.endTime) + + val inputBlobProcessingWindowInSec= inputBlobStartTime.until(inputBlobEndTime, ChronoUnit.SECONDS) + val inputBlobPartitionIncrementInSec = blobs.partitionIncrementInMin*60 getInputBlobPathPrefixes( - prefix = inputBlobPathPrefix, - datetimeFormat = inputBlobDateTimeFormat, + path = inputBlobPath, startTime = inputBlobStartTime, - durationInSeconds = inputBlobDurationInHours*3600, - timeIntervalInSeconds = inputBlobTimeIntervalInHours*3600 + inputBlobProcessingWindowInSec, + inputBlobPartitionIncrementInSec ) - }).par + }) val spark = appHost.getSpark(config.sparkConf) val sc = spark.sparkContext val processor = processorGenerator(config) - val ec = new ExecutionContext { - val threadPool = Executors.newFixedThreadPool(16) - def execute(runnable: Runnable) { - threadPool.submit(runnable) - } - def reportFailure(t: Throwable) {} - } - - prefixes.tasksupport = new ExecutionContextTaskSupport(ec) - - val batchResult = prefixes.map(prefix =>{ - appLog.warn(s"Start processing ${prefix}") - val namespace = "_"+HadoopClient.tempFilePrefix(prefix._1) - appLog.warn(s"Namespace for prefix ${prefix._1} is '$namespace'") - val pathsRDD = sc.makeRDD(HadoopClient.listFiles(prefix._1).toSeq) - val result = processor.processPathsRDD(pathsRDD, prefix._2, 1 hour, prefix._2, namespace) - appLog.warn(s"End processing ${prefix}") - - result - }).reduce(DataMerger.mergeMapOfDoubles) + val filesToProcess = prefixes.flatMap(prefix=>HadoopClient.listFiles(prefix._1).toSeq) + val minTimestamp = prefixes.minBy(_._2.getTime)._2 + appLog.warn(s"Start processing for $minTimestamp") + val pathsRDD = sc.makeRDD(filesToProcess) + val batchResult = processor.process(pathsRDD, minTimestamp, 1 hour) + appLog.warn(s"End processing for $minTimestamp") appLog.warn(s"Batch Mode Work Ended, processed metrics: $batchResult") AppInsightLogger.trackEvent(ProductConstant.ProductRoot + "/batch/end", null, batchResult) } - -} +} \ No newline at end of file diff --git a/DataProcessing/datax-host/src/main/scala/datax/host/SparkJarLoader.scala b/DataProcessing/datax-host/src/main/scala/datax/host/SparkJarLoader.scala index 3a325deac..cfdccde05 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/host/SparkJarLoader.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/host/SparkJarLoader.scala @@ -31,7 +31,8 @@ object SparkJarLoader { ScalaReflection.schemaFor(ts.baseType(udfInterface).typeArgs.last).dataType } - def addJarOnDriver(spark: SparkSession, jarPath: String, timestamp: Long = 0, resolveStorageKey:Boolean=true) = { + def addJarOnDriver(spark: SparkSession, jarPath: String, timestamp: Long = 0, resolveStorageKey:Boolean=true) : String = { + var fileUrl = "" val logger = LogManager.getLogger("AddJar") val localName = new URI(jarPath).getPath.split("/").last val currentTimeStamp = currentJars.get(jarPath) @@ -47,17 +48,22 @@ object SparkJarLoader { localName, resolveStorageKey) // Add it to our class loader - val url = new java.io.File(SparkFiles.getRootDirectory(), localName).toURI.toURL + val url = new java.io.File(SparkFiles.getRootDirectory(), localName).toURI.toURL + fileUrl = url.getPath() if (!ClassLoaderHost.urlClassLoader.getURLs().contains(url)) { logger.info("Adding " + url + " to class loader") ClassLoaderHost.urlClassLoader.addURL(url) } } + fileUrl } def addJar(spark: SparkSession, jarPath: String) = { - addJarOnDriver(spark, jarPath) - spark.sparkContext.addJar(jarPath) + val jarFileUrl = addJarOnDriver(spark, jarPath) + val logger = LogManager.getLogger("AddJar to executer") + logger.info("jarFileUrl is " + jarFileUrl) + // Add jar file to executers from the local path in driver node + spark.sparkContext.addJar(jarFileUrl) } def loadUdf(spark: SparkSession, udfName: String, jarPath: String, mainClass: String, method: String) = { diff --git a/DataProcessing/datax-host/src/main/scala/datax/input/BatchBlobInputSetting.scala b/DataProcessing/datax-host/src/main/scala/datax/input/BatchBlobInputSetting.scala index a9aaabfb7..3f36e7ae2 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/input/BatchBlobInputSetting.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/input/BatchBlobInputSetting.scala @@ -5,29 +5,44 @@ package datax.input import datax.config.{SettingDictionary, SettingNamespace} +import datax.securedsetting.KeyVaultClient import org.apache.log4j.LogManager -import scala.collection.mutable - -case class InputBlobsConf(pathPrefix:String, - pathPartitionFolderFormat: String, - startTime: String, - durationInHours: Long) +case class InputBlobsConf(path:String, + startTime: String, + endTime: String, + format: String, + compression: String, + partitionIncrementInMin:Long) object BatchBlobInputSetting { - val NamespaceBlobsSource = "blobs" - val NamespacePrefix = SettingNamespace.JobInputPrefix+NamespaceBlobsSource+"." + val NamespaceBlobsSource = "blob" + val NamespacePrefix = SettingNamespace.JobInputPrefix + NamespaceBlobsSource + SettingNamespace.Seperator + + val SettingPath = "path" + val SettingProcessStartTime = "processstarttime" + val SettingProcessEndTime = "processendtime" + val SettingPartitionIncrement = "partitionincrement" + val SettingFormat = "format" + val SettingCompression = "compression" + + val logger = LogManager.getLogger(this.getClass) private def buildInputBlobsConf(dict: SettingDictionary, name: String): InputBlobsConf = { + logger.warn("Load Dictionary from buildInputBlobsConf as following:\n"+dict.dict.map(kv=>s"${kv._1}->${kv._2}").mkString("\n")) + InputBlobsConf( - pathPrefix = dict.getOrNull("pathprefix"), - pathPartitionFolderFormat = dict.getOrNull("pathpartitionfolderformat"), - startTime = dict.getOrNull("starttime"), - durationInHours = dict.getLong("durationinhours") + path = KeyVaultClient.resolveSecretIfAny(dict.getOrNull(SettingPath)), + startTime = dict.getOrNull(SettingProcessStartTime), + endTime = dict.getOrNull(SettingProcessEndTime), + format = dict.getOrNull(SettingFormat), + compression = dict.getOrNull(SettingCompression), + partitionIncrementInMin = dict.getLong(SettingPartitionIncrement) ) } def getInputBlobsArrayConf(dict: SettingDictionary): Seq[InputBlobsConf] = { - dict.buildConfigIterable(buildInputBlobsConf, NamespacePrefix).toSeq + logger.warn("Blob namespace="+NamespacePrefix) + dict.buildConfigIterable(buildInputBlobsConf, NamespacePrefix).toSeq } } diff --git a/DataProcessing/datax-host/src/main/scala/datax/input/BlobPointerInput.scala b/DataProcessing/datax-host/src/main/scala/datax/input/BlobPointerInput.scala index 48038c474..f910a4128 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/input/BlobPointerInput.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/input/BlobPointerInput.scala @@ -11,6 +11,7 @@ import com.fasterxml.jackson.annotation.{JsonCreator, JsonProperty} import com.fasterxml.jackson.databind.ObjectMapper import com.microsoft.azure.eventhubs.EventData import datax.config._ +import datax.constants.BlobProperties import datax.data.FileInternal import datax.exception.EngineException import datax.input.BlobPointerInputSetting.BlobPointerInputConf @@ -38,7 +39,7 @@ object BlobPointerInput { (new StructType).add("BlobPath", StringType) } - private val saRegex = """wasbs?://[\w-]+@([\w\d]+)\.blob.core.windows.net/.*""".r + private val saRegex = s"""wasbs?://[\w-]+@([\w\d]+)\${BlobProperties.BlobHostPath}/.*""".r private def extractSourceId(blobPath: String, regex: String): String = { val r = if(regex == null) saRegex else regex.r r.findFirstMatchIn(blobPath) match { diff --git a/DataProcessing/datax-host/src/main/scala/datax/processor/BatchBlobProcessor.scala b/DataProcessing/datax-host/src/main/scala/datax/processor/BatchBlobProcessor.scala new file mode 100644 index 000000000..597987773 --- /dev/null +++ b/DataProcessing/datax-host/src/main/scala/datax/processor/BatchBlobProcessor.scala @@ -0,0 +1,14 @@ +package datax.processor + +import java.sql.Timestamp +import datax.utility.DateTimeUtil +import org.apache.spark.rdd.RDD + +import scala.concurrent.duration.Duration + +class BatchBlobProcessor(processBatchBlobPaths: (RDD[String], Timestamp, Duration, Timestamp, String) => Map[String, Double]) { + + val process = (rdd: RDD[String], batchTime: Timestamp, batchInterval: Duration) => { + processBatchBlobPaths(rdd, batchTime, batchInterval, batchTime, "") + } +} diff --git a/DataProcessing/datax-host/src/main/scala/datax/processor/CommonProcessor.scala b/DataProcessing/datax-host/src/main/scala/datax/processor/CommonProcessor.scala index 5a4859ca0..479ed1dfb 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/processor/CommonProcessor.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/processor/CommonProcessor.scala @@ -18,6 +18,7 @@ case class CommonProcessor( processJson: (RDD[String], Timestamp, Duration, Time processEventHubDataFrame: (DataFrame) => Map[String, StreamingQuery], processEventData: (RDD[EventData], Timestamp, Duration, Timestamp) => Map[String, Double], processPaths: (RDD[String], Timestamp, Duration, Timestamp, String) => Map[String, Double], + processBatchBlobPaths: (RDD[String], Timestamp, Duration, Timestamp, String) => Map[String, Double], processConsumerRecord: (RDD[ConsumerRecord[String,String]], Timestamp, Duration, Timestamp) => Map[String, Double]){ def asBlobPointerProcessor() = new BlobPointerProcessor(processPaths = this.processPaths) @@ -26,4 +27,5 @@ case class CommonProcessor( processJson: (RDD[String], Timestamp, Duration, Time def asStructuredStreamingProcessor = new EventHubStructuredStreamingProcessor(processDataFrame = this.processEventHubDataFrame) def asDirectLocalProcessor() = new DirectLocalProcessor(processEventData = this.processEventData) def asDirectKafkaProcessor() = new DirectKafkaProcessor(processConsumerRecord = this.processConsumerRecord) + def asBatchBlobProcessor() = new BatchBlobProcessor(processBatchBlobPaths = this.processBatchBlobPaths) } diff --git a/DataProcessing/datax-host/src/main/scala/datax/processor/CommonProcessorFactory.scala b/DataProcessing/datax-host/src/main/scala/datax/processor/CommonProcessorFactory.scala index d213f8fad..3767b86c7 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/processor/CommonProcessorFactory.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/processor/CommonProcessorFactory.scala @@ -8,6 +8,7 @@ import java.sql.Timestamp import java.util.concurrent.Executors import com.microsoft.azure.eventhubs.EventData +import datax.executor.ExecutorHelper import datax.config._ import datax.constants._ import datax.data.FileInternal @@ -15,10 +16,11 @@ import datax.exception.EngineException import datax.fs.HadoopClient import datax.host.{AppHost, CommonAppHost, SparkSessionSingleton, UdfInitializer} import datax.input.{BlobPointerInput, InputManager, SchemaFile, StreamingInputSetting} -import datax.sink.{OutputManager, OutputOperator} +import datax.sink.{BlobSinker, OutputManager, OutputOperator} import datax.telemetry.{AppInsightLogger, MetricLoggerFactory} import datax.utility._ import datax.handler._ +import datax.input.BlobPointerInput.{inputPathToInternalProps, pathHintsFromBlobPath} import datax.sql.TransformSQLParser import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.log4j.LogManager @@ -457,6 +459,60 @@ object CommonProcessorFactory { processDataset(jsonRdd.map((FileInternal(), _)).toDF(ColumnName.InternalColumnFileInfo, ColumnName.RawObjectColumn), batchTime, batchInterval, outputPartitionTime, null, "") }, + // process blob path from batch blob input + processBatchBlobPaths = (pathsRDD: RDD[String], + batchTime: Timestamp, + batchInterval: Duration, + outputPartitionTime: Timestamp, + namespace: String) => { + + + val spark = SparkSessionSingleton.getInstance(pathsRDD.sparkContext.getConf) + + val metricLogger = MetricLoggerFactory.getMetricLogger(metricAppName, metricConf) + val batchTimeStr = DateTimeUtil.formatSimple(batchTime) + val batchLog = LogManager.getLogger(s"BatchProcessor-B$batchTimeStr") + val batchTimeInMs = batchTime.getTime + + def postMetrics(metrics: Iterable[(String, Double)]): Unit = { + metricLogger.sendBatchMetrics(metrics, batchTime.getTime) + batchLog.warn(s"Metric ${metrics.map(m => m._1 + "=" + m._2).mkString(",")}") + } + + batchLog.warn(s"Start batch ${batchTime}, output partition time:${outputPartitionTime}, namespace:${namespace}") + val t1 = System.nanoTime + + // Wrap files to FileInternal object + val internalFiles = pathsRDD.map(file => { + FileInternal(inputPath = file, + outputFolders = null, + outputFileName = null, + fileTime = null, + ruleIndexPrefix = "", + target = null + ) + }) + + val paths = internalFiles.map(_.inputPath).collect() + postMetrics(Map(s"InputBlobs" -> paths.size.toDouble)) + batchLog.warn(s"InputBlob count=${paths.size}"); + + val blobStorageKey = ExecutorHelper.createBlobStorageKeyBroadcastVariable(paths.head, spark) + + val inputDf = internalFiles + .flatMap(file => HadoopClient.readHdfsFile(file.inputPath, gzip = file.inputPath.endsWith(".gz"), blobStorageKey) + .filter(l=>l!=null && !l.isEmpty).map((file, outputPartitionTime, _))) + .toDF(ColumnName.InternalColumnFileInfo, ColumnName.MetadataColumnOutputPartitionTime, ColumnName.RawObjectColumn) + + val processedMetrics = processDataset(inputDf, batchTime, batchInterval, outputPartitionTime, null, "") + + val batchProcessingTime = (System.nanoTime - t1) / 1E9 + + val metrics = Map[String, Double]( + "BatchProcessedET" -> batchProcessingTime + ) + processedMetrics ++ metrics + }, // process blob path pointer data frame processPaths = (pathsRDD: RDD[String], diff --git a/DataProcessing/datax-host/src/main/scala/datax/securedsetting/KeyVaultClient.scala b/DataProcessing/datax-host/src/main/scala/datax/securedsetting/KeyVaultClient.scala index f3920c346..4a91823ae 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/securedsetting/KeyVaultClient.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/securedsetting/KeyVaultClient.scala @@ -9,7 +9,7 @@ import datax.constants.JobArgument import datax.exception.EngineException import datax.keyvault.KeyVaultMsiAuthenticatorClient import org.apache.log4j.LogManager - +import com.databricks.dbutils_v1.DBUtilsHolder.dbutils import scala.collection.mutable @@ -18,7 +18,7 @@ import scala.collection.mutable */ object KeyVaultClient { private val logger = LogManager.getLogger(this.getClass) - private val secretRegex = "^keyvault:\\/\\/([a-zA-Z0-9-_]+)\\/([a-zA-Z0-9-_]+)$".r + private val secretRegex = "^(keyvault|secretscope):\\/\\/([a-zA-Z0-9-_]+)\\/([a-zA-Z0-9-_]+)$".r private val kvc = KeyVaultMsiAuthenticatorClient.getKeyVaultClient() private val cache = new mutable.HashMap[String, String] @@ -33,19 +33,27 @@ object KeyVaultClient { return Option(secretId) secretRegex.findFirstMatchIn(secretId) match { - case Some(secretInfo) => val vaultName = secretInfo.group(1) - val secretName = secretInfo.group(2) + case Some(secretInfo) => val secretType = secretInfo.group(1) + val vaultName = secretInfo.group(2) + val secretName = secretInfo.group(3) cache.synchronized{ cache.get(secretId) match { case Some(value) => Some(value) case None => - val secret = kvc.synchronized{ - kvc.getSecret(s"https://$vaultName.vault.azure.net",secretName) - } + var value = "" + if(secretType == "secretscope"){ + value = dbutils.synchronized{ + dbutils.secrets.get(scope = vaultName, key = secretName) + } + } + else{ + value = kvc.synchronized{ + kvc.getSecret(s"https://$vaultName.vault.azure.net",secretName) + }.value() + } logger.warn(s"resolved secret:'$secretId'") - val value = secret.value() cache(secretId) = value Some(value) } diff --git a/DataProcessing/datax-host/src/main/scala/datax/sink/BlobSinker.scala b/DataProcessing/datax-host/src/main/scala/datax/sink/BlobSinker.scala index 44a03c799..2387187f4 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/sink/BlobSinker.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/sink/BlobSinker.scala @@ -8,6 +8,7 @@ package datax.sink import java.sql.Timestamp import java.text.SimpleDateFormat +import datax.executor.ExecutorHelper import datax.config._ import datax.constants.{JobArgument, MetricName} import datax.data.{FileInternal, ProcessResult} @@ -15,12 +16,15 @@ import datax.fs.HadoopClient import datax.securedsetting.KeyVaultClient import datax.sink.BlobOutputSetting.BlobOutputConf import datax.utility.{GZipHelper, SinkerUtil} +import datax.constants.{ProductConstant, BlobProperties} +import datax.config.ConfigManager +import datax.host.SparkSessionSingleton +import org.apache.spark.broadcast import org.apache.log4j.LogManager import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.{DataFrame, Row, SparkSession} -import scala.collection.mutable import scala.concurrent.duration.Duration object BlobSinker extends SinkOperatorFactory { @@ -47,7 +51,7 @@ object BlobSinker extends SinkOperatorFactory { } // write events to blob location - def writeEventsToBlob(data: Seq[String], outputPath: String, compression: Boolean) { + def writeEventsToBlob(data: Seq[String], outputPath: String, compression: Boolean, blobStorageKey: broadcast.Broadcast[String] = null) { val logger = LogManager.getLogger(s"EventsToBlob-Writer${SparkEnvVariables.getLoggerSuffix()}") val countEvents = data.length @@ -76,7 +80,8 @@ object BlobSinker extends SinkOperatorFactory { hdfsPath = outputPath, content = content, timeout = Duration.create(ConfigManager.getActiveDictionary().getOrElse(JobArgument.ConfName_BlobWriterTimeout, "10 seconds")), - retries = 0 + retries = 0, + blobStorageKey ) timeNow = System.nanoTime() logger.info(s"$timeNow:Step 3: done writing to $outputPath, spent time=${(timeNow - timeLast) / 1E9} seconds") @@ -127,7 +132,8 @@ object BlobSinker extends SinkOperatorFactory { outputFolders: Map[String, String], partitionId: Int, compression: Boolean, - loggerSuffix: String): Map[String, Int] = { + loggerSuffix: String, + blobStorageKey: broadcast.Broadcast[String]): Map[String, Int] = { val logger = LogManager.getLogger(s"Sinker-BlobSinker$loggerSuffix") val dataGroups = dataGenerator() val timeStart = System.nanoTime () @@ -141,7 +147,7 @@ object BlobSinker extends SinkOperatorFactory { val path = folder + (if (fileName == null) s"part-$partitionId" else fileName) + (if(compression) ".json.gz" else ".json") val jsonList = data.toSeq - BlobSinker.writeEventsToBlob(jsonList, path, compression ) + BlobSinker.writeEventsToBlob(jsonList, path, compression, blobStorageKey ) Seq(s"${MetricPrefixEvents}$group" -> jsonList.length, s"${MetricPrefixBlobs}$group" -> 1) } @@ -158,7 +164,10 @@ object BlobSinker extends SinkOperatorFactory { if(formatConf.isDefined && !formatConf.get.equalsIgnoreCase("json")) throw new Error(s"Output format: ${formatConf.get} as specified in the config is not supported") val outputFolders = blobOutputConf.groups.map{case(k,v)=>k->KeyVaultClient.resolveSecretIfAny(v.folder)} - (rowInfo: Row, rows: Seq[Row], outputPartitionTime: Timestamp, partitionId: Int, loggerSuffix: String) => { + + val blobStorageKey = ExecutorHelper.createBlobStorageKeyBroadcastVariable(outputFolders.head._2, SparkSessionSingleton.getInstance(ConfigManager.initSparkConf)) + + val jsonSinkDelegate = (rowInfo: Row, rows: Seq[Row], outputPartitionTime: Timestamp, partitionId: Int, loggerSuffix: String) => { val target = FileInternal.getInfoTargetTag(rowInfo) if(compressionTypeConf.isDefined && !(compressionTypeConf.get.equalsIgnoreCase("gzip")|| compressionTypeConf.get.equalsIgnoreCase("none")|| compressionTypeConf.get.equals(""))) throw new Error(s"Output compressionType: ${compressionTypeConf.get} as specified in the config is not supported") @@ -175,9 +184,14 @@ object BlobSinker extends SinkOperatorFactory { k->generateOutputFolderPath(v, outputPartitionTime, Option(target))}, partitionId = partitionId, compression = compression, - loggerSuffix = loggerSuffix + loggerSuffix = loggerSuffix, + blobStorageKey = blobStorageKey ) } + + (data: DataFrame, time: Timestamp, loggerSuffix: String) => { + SinkerUtil.sinkJson(data, time, jsonSinkDelegate) + } } def getSinkOperator(dict: SettingDictionary, name: String): SinkOperator = { @@ -185,6 +199,7 @@ object BlobSinker extends SinkOperatorFactory { SinkOperator( name = SinkName, isEnabled = blobConf!=null, + sinkAsJson = true, flagColumnExprGenerator = () => blobConf.groupEvaluation.getOrElse(null), generator = flagColumnIndex=>getRowsSinkerGenerator(blobConf, flagColumnIndex), onBatch = (spark: SparkSession, outputPartitionTime: Timestamp, targets: Set[String]) => { diff --git a/DataProcessing/datax-host/src/main/scala/datax/sink/CosmosDBSinker.scala b/DataProcessing/datax-host/src/main/scala/datax/sink/CosmosDBSinker.scala index 879efa99f..9ab7a6b53 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/sink/CosmosDBSinker.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/sink/CosmosDBSinker.scala @@ -120,6 +120,7 @@ object CosmosDBSinkerManager extends SinkOperatorFactory { SinkOperator( name = SinkName, isEnabled = conf!=null, + sinkAsJson = true, flagColumnExprGenerator = () => null, generator = flagColumnIndex => SinkerUtil.outputGenerator( (dataToSend:Seq[String],ls: String) => { diff --git a/DataProcessing/datax-host/src/main/scala/datax/sink/EventHubStreamPoster.scala b/DataProcessing/datax-host/src/main/scala/datax/sink/EventHubStreamPoster.scala index 4af727bc2..2ce5c09a5 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/sink/EventHubStreamPoster.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/sink/EventHubStreamPoster.scala @@ -72,6 +72,7 @@ object EventHubStreamPoster extends SinkOperatorFactory { SinkOperator( name = SinkName, isEnabled = conf!=null, + sinkAsJson = true, flagColumnExprGenerator = () => conf.filter, generator = flagColumnIndex => getRowsSinkerGenerator(conf, flagColumnIndex) ) diff --git a/DataProcessing/datax-host/src/main/scala/datax/sink/HttpPoster.scala b/DataProcessing/datax-host/src/main/scala/datax/sink/HttpPoster.scala index 8b65da094..69f876015 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/sink/HttpPoster.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/sink/HttpPoster.scala @@ -75,6 +75,7 @@ object HttpPoster extends SinkOperatorFactory { SinkOperator( name = SinkName, isEnabled = conf!=null, + sinkAsJson = true, flagColumnExprGenerator = () => conf.filter, generator = (flagColumnIndex)=>getRowsSinkerGenerator(conf, flagColumnIndex) ) diff --git a/DataProcessing/datax-host/src/main/scala/datax/sink/OutputManager.scala b/DataProcessing/datax-host/src/main/scala/datax/sink/OutputManager.scala index 75df86ebe..910ce0916 100644 --- a/DataProcessing/datax-host/src/main/scala/datax/sink/OutputManager.scala +++ b/DataProcessing/datax-host/src/main/scala/datax/sink/OutputManager.scala @@ -24,7 +24,7 @@ object OutputManager { val SettingOutputProcessedSchemaPath = "processedschemapath" val sinkFactories = Seq[SinkOperatorFactory]( - BlobSinker, EventHubStreamPoster, HttpPoster, CosmosDBSinkerManager + BlobSinker, EventHubStreamPoster, HttpPoster, CosmosDBSinkerManager, SqlSinker ).map(f=>f.getSettingNamespace()->f).toMap def getOperatators(dict: SettingDictionary): Seq[OutputOperator] ={ @@ -41,33 +41,33 @@ object OutputManager { val processedSchemaPath = dict.get(SettingOutputProcessedSchemaPath).orNull val sinkOperators = dict - .groupBySubNamespace() - .map { case (k, v) => sinkFactories.get(k).map(_.getSinkOperator(v, k))} + .groupBySubNamespace() + .map { case (k, v) => sinkFactories.get(k).map(_.getSinkOperator(v, k))} .filter(o => o match { case Some(oper) => logger.info(s"Output '$name':${oper.name} is ${SinkerUtil.boolToOnOff(oper.isEnabled)}") oper.isEnabled case None => false }).map(o=>{ - val oper = o.get - val flagColumnExpr = oper.flagColumnExprGenerator() - if(flagColumnExpr==null){ - logger.warn(s"Output type:'${oper.name}': no flag column") - (oper.name, (null, null), oper.generator(-1), oper.onBatch, oper.onInitialization) - } - else{ - val appendColumn = (flagColumnExpr, s"_${ProductConstant.ProductOutputFilter}_${oper.name}") - logger.warn(s"Output type:'${oper.name}': append column:$appendColumn") - flagColumnIndex+=1 - (oper.name, appendColumn, oper.generator(flagColumnIndex), oper.onBatch, oper.onInitialization) - } - }).toSeq + val oper = o.get + val flagColumnExpr = oper.flagColumnExprGenerator() + if(flagColumnExpr==null){ + logger.warn(s"Output type:'${oper.name}': no flag column") + (oper.name, (null, null), oper.generator(-1), oper.onBatch, oper.onInitialization, oper.sinkAsJson) + } + else{ + val appendColumn = (flagColumnExpr, s"_${ProductConstant.ProductOutputFilter}_${oper.name}") + logger.warn(s"Output type:'${oper.name}': append column:$appendColumn") + flagColumnIndex+=1 + (oper.name, appendColumn, oper.generator(flagColumnIndex), oper.onBatch, oper.onInitialization, oper.sinkAsJson) + } + }).toSeq if(sinkOperators.length==0)throw new EngineException(s"no sink is defined for output '$name'!") logger.warn(s"Output '$name' to ${sinkOperators.length} sinkers: ${sinkOperators.map(s=>s"'${s._1}'").mkString(",")}") val flagColumns = sinkOperators.map(_._2).filter(_._1!=null) - val sinkers = sinkOperators.map(o=>o._1 -> o._3).toMap + val sinkers = sinkOperators.map(o=>(o._1, o._3, o._6)) val onBatchHandlers = sinkOperators.map(_._4).filter(_!=null) val onInitHandlers = sinkOperators.map(_._5).filter(_!=null) var shouldGeneratorProcessedSchema = processedSchemaPath!=null && !processedSchemaPath.isEmpty @@ -95,60 +95,67 @@ object OutputManager { val outputColumnNames = outputColumns.map(c=>DataNormalization.sanitizeColumnName(c.name)) outputLogger.warn(s"Output fields: ${outputColumnNames.mkString(",")}") - sink(df, outputColumnNames, partitionTime, flagColumns, sinkers) + sink(name, df, outputColumnNames, partitionTime, flagColumns, sinkers) } ) } - def sink(df: DataFrame, + def sink(sinkerName:String, + df: DataFrame, outputFieldNames: Seq[String], partitionTime: Timestamp, flagColumns: Seq[(String, String)], - outputOperators: Map[String, SinkDelegate]) = { - val amendDf = if(df.schema.fieldNames.contains(ColumnName.InternalColumnFileInfo))df + outputOperators: Seq[(String, SinkDelegate, Boolean)]) = { + + // Create json'ified dataframe for sinking to outputs that expect json data + val amendDf = if (df.schema.fieldNames.contains(ColumnName.InternalColumnFileInfo)) df else { df.withColumn(ColumnName.InternalColumnFileInfo, FileInternal.udfEmptyInternalInfo()) } - val query = amendDf.selectExpr("*" +: flagColumns.map(c => c._1 + " AS " + c._2): _*) + val jsonDf = amendDf.selectExpr("*" +: flagColumns.map(c => c._1 + " AS " + c._2): _*) .select(Seq(col(ColumnName.InternalColumnFileInfo), to_json(struct(outputFieldNames.map(col): _*))) ++ flagColumns.map(_._2).map(col): _*) - //query.explain will dump the execution plan of sql to stdout - //query.explain(true) - query - .rdd - .mapPartitions(it => { - val partitionId = TaskContext.getPartitionId() - val loggerSuffix = SparkEnvVariables.getLoggerSuffix() - val logger = LogManager.getLogger(s"EventsSinker${loggerSuffix}") - //val path = outputFileFolder+"/part-"+tc.partitionId().toString + ".json.gz" - - val t1 = System.nanoTime() - var timeLast = t1 - var timeNow: Long = 0 - logger.info(s"$timeNow:Partition started") - - val dataAll = it.toArray - val count = dataAll.length - timeNow = System.nanoTime() - logger.info(s"$timeNow:Collected $count events, spent time=${(timeNow - timeLast) / 1E9} seconds") - timeLast = timeNow - - val inputMetric = Map(s"${MetricName.MetricSinkPrefix}InputEvents" -> count) - Seq(if (count > 0) { - val rowInfo = dataAll(0).getAs[Row](0) - if(outputOperators.size==0) - throw new EngineException("no output operators are found!") - outputOperators - .par - .map(_._2(rowInfo, dataAll, partitionTime, partitionId, loggerSuffix)) - .reduce(DataMerger.mergeMapOfCounts) ++ inputMetric + + val count = df.count().toInt + val inputMetric = Map(s"${MetricName.MetricSinkPrefix}InputEvents" -> count) + + val logger = LogManager.getLogger(s"EventsSinker-${sinkerName}") + logger.warn(s"Dataframe sinker ${sinkerName} started to sink ${count} events") + + + // Get the output operators that sink JSON and non-JSON data and call the output sinkers in parallel + val jsonOutputOperators = outputOperators.filter(_._3) + val nonJsonOutputOperators = outputOperators.filter(!_._3) + + if (count > 0) { + Seq("json", "nonjson").par.map { + case "json" => + if (jsonOutputOperators.size > 0) { + jsonOutputOperators + .par + .map(_._2(jsonDf, partitionTime, sinkerName)) + .reduce(DataMerger.mergeMapOfCounts) + } + else { + Map.empty[String,Int] + } + + case "nonjson" => { + if (nonJsonOutputOperators.size > 0) { + nonJsonOutputOperators + .par + .map(_._2(df, partitionTime, sinkerName)) + .reduce(DataMerger.mergeMapOfCounts) + } + else { + Map.empty[String,Int] + } } - else - inputMetric - ).iterator - }) - .reduce(DataMerger.mergeMapOfCounts) + }.reduce(DataMerger.mergeMapOfCounts) ++ inputMetric + } + else + inputMetric } -} +} \ No newline at end of file diff --git a/DataProcessing/datax-host/src/main/scala/datax/sink/SqlOutputSetting.scala b/DataProcessing/datax-host/src/main/scala/datax/sink/SqlOutputSetting.scala new file mode 100644 index 000000000..cdf52c918 --- /dev/null +++ b/DataProcessing/datax-host/src/main/scala/datax/sink/SqlOutputSetting.scala @@ -0,0 +1,61 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +package datax.sink + +import datax.client.sql.SqlConf +import datax.config.SettingDictionary +import datax.securedsetting.KeyVaultClient + + +object SqlOutputSetting { + val Namespace = "sql" + val SettingConnectionString = "connectionstring" + val SettingUrl="url" + val SettingEncrypt = "encrypt" + val SettingTrustServerCertificate = "trustServerCertificate" + val SettingHostNameInCertificate = "hostNameInCertificate" + val SettingDatabaseName = "databasename" + val SettingTable = "table" + val SettingFilter="filter" + val SettingWriteMode="writemode" + val SettingUserName = "user" + val SettingPassword = "password" + val SettingConnectionTimeout = "connectiontimeout" + val SettingQueryTimeout = "querytimeout" + val SettingUseBulkCopy = "usebulkinsert" + val SettingUseBulkCopyTablelock = "usebulkcopytablelock" + val SettingUseBulkCopyInternalTransaction = "usebulkcopyinternaltransaction" + val SettingUseBulkCopyTimeout = "bulkcopytimeout" + val SettingUseBulkBatchSize = "bulkcopybatchsize" + + def buildSqlOutputConf(dict: SettingDictionary, name: String): SqlConf = { + KeyVaultClient.resolveSecretIfAny(dict.get(SettingConnectionString)) match { + case Some(connectionString) => + SqlConf( + name = name, + connectionString = connectionString, + url = KeyVaultClient.resolveSecretIfAny(dict.getOrNull(SettingUrl)), + encrypt = dict.getOrNull(SettingEncrypt), + trustServerCertificate = dict.getOrNull(SettingTrustServerCertificate), + hostNameInCertificate = dict.getOrNull(SettingHostNameInCertificate), + databaseName = dict.getOrNull(SettingDatabaseName), + table = dict.getOrNull(SettingTable), + writeMode = dict.getOrElse(SettingWriteMode,"append"), + userName = KeyVaultClient.resolveSecretIfAny(dict.getString(SettingUserName)), + password = KeyVaultClient.resolveSecretIfAny(dict.getString(SettingPassword)), + filter = dict.getOrNull(SettingFilter), + connectionTimeout=dict.getOrElse(SettingConnectionTimeout,"30"), + queryTimeout = dict.getOrElse(SettingQueryTimeout,"30"), + useBulkCopy = dict.getOrElse(SettingUseBulkCopy,"false").toBoolean, + useBulkCopyTableLock = dict.getOrElse(SettingUseBulkCopyTablelock,"false"), + useBulkCopyInternalTransaction = dict.getOrElse(SettingUseBulkCopyInternalTransaction,"false"), + bulkCopyTimeout = dict.getOrElse(SettingUseBulkCopyTimeout,"60"), + bulkCopyBatchSize = dict.getOrElse(SettingUseBulkBatchSize,"2000") + ) + case None => null + } + } + +} diff --git a/DataProcessing/datax-host/src/main/scala/datax/sink/SqlSinker.scala b/DataProcessing/datax-host/src/main/scala/datax/sink/SqlSinker.scala new file mode 100644 index 000000000..775ceae41 --- /dev/null +++ b/DataProcessing/datax-host/src/main/scala/datax/sink/SqlSinker.scala @@ -0,0 +1,107 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +package datax.sink + +import datax.client.sql.SqlConf +import datax.config.SettingDictionary +import datax.utility.SinkerUtil +import org.apache.spark.sql.{DataFrame, Row} +import com.microsoft.azure.sqldb.spark.config.Config +import com.microsoft.azure.sqldb.spark.connect._ + +object SqlSinker extends SinkOperatorFactory { + + val SinkName = "Sql" + + private def writeToSql(dataToSend:DataFrame, sqlConf: SqlConf, logSuffix:String) = { + + if(sqlConf.useBulkCopy) { + writeUsingSqlBulkCopy(dataToSend,sqlConf,logSuffix) + } + else { + writeUsingSqlConnector(dataToSend,sqlConf,logSuffix) + } + } + + private def writeUsingSqlJdbc(dataToSend:DataFrame, sqlConf: SqlConf, logSuffix:String) ={ + + val sqlurl = sqlConf.connectionString + val prop = new java.util.Properties + prop.setProperty("driver", "com.microsoft.sqlserver.jdbc.SQLServerDriver") + + dataToSend.write.mode(sqlConf.writeMode).jdbc(sqlurl, sqlConf.table, prop) + dataToSend.count().toInt + } + + private def writeUsingSqlConnector(dataToSend:DataFrame, sqlConf: SqlConf, logSuffix:String) ={ + + val cfgMap = getConfigMap(sqlConf) + dataToSend.write.mode(sqlConf.writeMode).sqlDB(Config(cfgMap)) + dataToSend.count().toInt + } + + + private def writeUsingSqlBulkCopy(dataToSend:DataFrame, sqlConf: SqlConf, logSuffix:String) ={ + + val bulkCopyConfig = getConfigMap(sqlConf) + dataToSend.bulkCopyToSqlDB(Config(bulkCopyConfig)) + dataToSend.count().toInt + } + + + private def getConfigMap(sqlConf: SqlConf): Map[String,String] = { + + var configMap = Map( + "url" -> sqlConf.url, + "databaseName" -> sqlConf.databaseName, + "user" -> sqlConf.userName, + "password" -> sqlConf.password, + "dbTable" -> sqlConf.table, + "connectTimeout" -> sqlConf.connectionTimeout, + "queryTimeout" -> sqlConf.queryTimeout + ) + + // encrypt and cert related settings goes hand in hand, all are needed to be set together + if(sqlConf.encrypt!=null && sqlConf.trustServerCertificate!=null && sqlConf.hostNameInCertificate!=null) + { + configMap += ( + "encrypt" -> sqlConf.encrypt, + "trustServerCertificate" -> sqlConf.trustServerCertificate, + "hostNameInCertificate" -> sqlConf.hostNameInCertificate) + } + + // if bulkCopy is enabled, add the corresponding settings + if(sqlConf.useBulkCopy) { + + configMap += ( + "bulkCopyBatchSize" -> sqlConf.bulkCopyBatchSize, + "bulkCopyTableLock" -> sqlConf.useBulkCopyTableLock, + "bulkCopyTimeout" -> sqlConf.bulkCopyTimeout, + "bulkCopyUseInternalTransaction" -> sqlConf.useBulkCopyInternalTransaction) + } + + configMap + } + + def getSinkOperator(dict: SettingDictionary, name: String) : SinkOperator = { + + val conf = SqlOutputSetting.buildSqlOutputConf(dict, name) + SinkOperator( + name = SinkName, + isEnabled = conf!=null, + sinkAsJson = false, + flagColumnExprGenerator = () => conf.filter, + generator = (flagColumnIndex)=> getRowsSinkerGeneratorDf(conf) + ) + } + + def getRowsSinkerGeneratorDf(sqlConf: SqlConf) : SinkDelegate = { + val sender = (dataToSend:DataFrame, loggerSuffix:String) => writeToSql(dataToSend, sqlConf, SinkName) + SinkerUtil.outputGenerator(sender,SinkName)() + } + + override def getSettingNamespace(): String = SqlOutputSetting.Namespace +} diff --git a/DataProcessing/datax-keyvault/pom.xml b/DataProcessing/datax-keyvault/pom.xml index e9f991aac..a7a02958e 100644 --- a/DataProcessing/datax-keyvault/pom.xml +++ b/DataProcessing/datax-keyvault/pom.xml @@ -50,11 +50,11 @@ SOFTWARE com.microsoft.datax - datax-keyvault_2.3_2.11 - 1.1.0 + datax-keyvault_2.4_2.11 + 1.2.0 - 2.3.0 + 2.4.0 2.11 8 ${scala.version.major}.${scala.version.minor} @@ -100,7 +100,7 @@ SOFTWARE com.fasterxml.jackson.core jackson-databind - 2.9.9 + 2.9.10 @@ -244,4 +244,4 @@ SOFTWARE - \ No newline at end of file + diff --git a/DataProcessing/datax-udf-samples/pom.xml b/DataProcessing/datax-udf-samples/pom.xml index c112c36dc..7ec7d897f 100644 --- a/DataProcessing/datax-udf-samples/pom.xml +++ b/DataProcessing/datax-udf-samples/pom.xml @@ -50,11 +50,11 @@ SOFTWARE com.microsoft.datax - datax-udf-samples_2.3_2.11 - 1.1.0 + datax-udf-samples_2.4_2.11 + 1.2.0 - 2.3.0 + 2.4.0 2.11 8 ${scala.version.major}.${scala.version.minor} @@ -112,8 +112,8 @@ SOFTWARE com.microsoft.datax - datax-core_2.3_2.11 - 1.1.0 + datax-core_2.4_2.11 + 1.2.0 diff --git a/DataProcessing/datax-utility/pom.xml b/DataProcessing/datax-utility/pom.xml index 6d99c20b7..9a9a3886f 100644 --- a/DataProcessing/datax-utility/pom.xml +++ b/DataProcessing/datax-utility/pom.xml @@ -50,11 +50,11 @@ SOFTWARE com.microsoft.datax - datax-utility_2.3_2.11 - 1.1.0 + datax-utility_2.4_2.11 + 1.2.0 - 2.3.0 + 2.4.0 2.11 8 ${scala.version.major}.${scala.version.minor} @@ -119,8 +119,8 @@ SOFTWARE com.microsoft.datax - datax-core_2.3_2.11 - 1.1.0 + datax-core_2.4_2.11 + 1.2.0 com.microsoft.azure @@ -129,8 +129,8 @@ SOFTWARE com.microsoft.datax - datax-keyvault_2.3_2.11 - 1.1.0 + datax-keyvault_2.4_2.11 + 1.2.0 com.microsoft.azure diff --git a/DataProcessing/datax-utility/src/main/scala/datax/utility/SinkerUtil.scala b/DataProcessing/datax-utility/src/main/scala/datax/utility/SinkerUtil.scala index 26ea3c13f..7be78ab47 100644 --- a/DataProcessing/datax-utility/src/main/scala/datax/utility/SinkerUtil.scala +++ b/DataProcessing/datax-utility/src/main/scala/datax/utility/SinkerUtil.scala @@ -6,10 +6,13 @@ package datax.utility import java.sql.Timestamp +import datax.config.SparkEnvVariables import datax.constants.MetricName +import datax.sink.JsonSinkDelegate import org.apache.commons.codec.digest.DigestUtils import org.apache.log4j.LogManager -import org.apache.spark.sql.Row +import org.apache.spark.TaskContext +import org.apache.spark.sql.{DataFrame, Row} object SinkerUtil { @@ -47,14 +50,75 @@ object SinkerUtil { Map(s"${MetricName.MetricSinkPrefix}${sinkerName}_All" -> resultCount) } + private def outputAllEvents(dataToSend: DataFrame, writer: (DataFrame, String)=>Int, sinkerName: String, loggerSuffix: String) = { + + val loggerPrefix = s"Sinker-$sinkerName-All" + val logger = LogManager.getLogger(loggerPrefix+loggerSuffix) + val timeStart = System.nanoTime() + + val resultCount = writer(dataToSend, loggerSuffix) + + val timeNow = System.nanoTime() + logger.info(s"$timeNow:written count= $resultCount, spent time=${(timeNow - timeStart) / 1E9} seconds") + + Map(s"${MetricName.MetricSinkPrefix}${sinkerName}_All" -> resultCount) + } + + def outputGenerator(writer: (DataFrame, String)=>Int, sinkerName: String) = { + + () => { + (data: DataFrame, time: Timestamp, loggerSuffix: String) => + outputAllEvents(data, writer, sinkerName, loggerSuffix) + } + } + def outputGenerator(writer: (Seq[String], String)=>Int, sinkerName: String) = { (flagColumnIndex: Int) => { - if(flagColumnIndex<0) - (rowInfo: Row, rows: Seq[Row], partitionTime: Timestamp, partitionId: Int, loggerSuffix: String) => - outputAllEvents(rows,writer, sinkerName, loggerSuffix) - else - (rowInfo: Row, rows: Seq[Row], partitionTime: Timestamp, partitionId: Int, loggerSuffix: String) => - SinkerUtil.outputFilteredEvents(rows, flagColumnIndex, writer, sinkerName, loggerSuffix) + if (flagColumnIndex < 0) { + (data: DataFrame, time: Timestamp, loggerSuffix: String) => { + sinkJson(data, time, (rowInfo: Row, rows: Seq[Row], partitionTime: Timestamp, partitionId: Int, loggerSuffix: String) => + outputAllEvents(rows, writer, sinkerName, loggerSuffix)) + } + } + else { + (data: DataFrame, time: Timestamp, loggerSuffix: String) => { + sinkJson(data, time, (rowInfo: Row, rows: Seq[Row], partitionTime: Timestamp, partitionId: Int, loggerSuffix: String) => + outputFilteredEvents(rows, flagColumnIndex, writer, sinkerName, loggerSuffix)) + } + } } } -} + + // Convert dataframe to sequence of rows and sink using the passed in sinker delegate. The rows contain data as json column + def sinkJson(df:DataFrame, partitionTime: Timestamp, jsonSinkDelegate:JsonSinkDelegate ): Map[String, Int] = { + + df + .rdd + .mapPartitions(it => { + val partitionId = TaskContext.getPartitionId() + val loggerSuffix = SparkEnvVariables.getLoggerSuffix() + val logger = LogManager.getLogger(s"EventsSinker${loggerSuffix}") + + val t1 = System.nanoTime() + var timeLast = t1 + var timeNow: Long = 0 + logger.info(s"$timeNow:Partition started") + + val dataAll = it.toArray + val count = dataAll.length + timeNow = System.nanoTime() + logger.info(s"$timeNow:Collected $count events, spent time=${(timeNow - timeLast) / 1E9} seconds") + timeLast = timeNow + + val inputMetric = Map(s"${MetricName.MetricSinkPrefix}InputEvents" -> count) + Seq(if (count > 0) { + val rowInfo = dataAll(0).getAs[Row](0) + jsonSinkDelegate(rowInfo, dataAll, partitionTime, partitionId, loggerSuffix) ++ inputMetric + } + else + inputMetric + ).iterator + }) + .reduce(DataMerger.mergeMapOfCounts) + } +} \ No newline at end of file diff --git a/DataProcessing/deploy.cmd b/DataProcessing/deploy.cmd index 168491baa..6fdab5460 100644 --- a/DataProcessing/deploy.cmd +++ b/DataProcessing/deploy.cmd @@ -3,7 +3,7 @@ @FOR /F "TOKENS=1,2 DELIMS=/ eol=/" %%A IN ('echo %CDATE%') DO @SET dd=%%B @FOR /F "TOKENS=2,3 DELIMS=/ " %%A IN ('echo %CDATE%') DO @SET yyyy=%%B SET date=%yyyy%%mm%%dd% -SET dataxver=1.1.0 +SET dataxver=1.2.0 SET module_name=%1 SET dst_name=%2 diff --git a/DeploymentCloud/Deployment.Common/Admin/adminsteps.parameters.txt b/DeploymentCloud/Deployment.Common/Admin/adminsteps.parameters.txt index ca25ccc57..a0e159707 100644 --- a/DeploymentCloud/Deployment.Common/Admin/adminsteps.parameters.txt +++ b/DeploymentCloud/Deployment.Common/Admin/adminsteps.parameters.txt @@ -10,4 +10,8 @@ serviceAppId=$serviceAppId clientAppId=$clientAppId # ResourceGroupName to generate resources -resourceGroupName=$resourceGroup \ No newline at end of file +resourceGroupName=$resourceGroup + +# Role Names +# Writer can manage flows +writerRole=$writerRole \ No newline at end of file diff --git a/DeploymentCloud/Deployment.Common/CosmosDB/commons.json b/DeploymentCloud/Deployment.Common/CosmosDB/commons.json index 2c33c37e6..3c4dca0d9 100644 --- a/DeploymentCloud/Deployment.Common/CosmosDB/commons.json +++ b/DeploymentCloud/Deployment.Common/CosmosDB/commons.json @@ -6,7 +6,7 @@ "icon": "/img/iot.png", "displayName": "###REQUIRED###", "properties": { - "inputEventHubSubscriptionId": "keyvault://${serviceKeyVaultName}/$clientSecretPrefix-subscriptionId", + "inputEventHubSubscriptionId": "$keyvaultPrefix://${serviceKeyVaultName}/$clientSecretPrefix-subscriptionId", "inputEventHubResourceGroupName": "${eventHubResourceGroupName}" }, "commonProcessor": { @@ -31,7 +31,7 @@ }, "process": { "metric": { - "eventhub": "keyvault://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" + "eventhub": "$keyvaultPrefix://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" }, "timestampColumn": "${processTimestampColumn}", "watermark": "${processWatermark}", @@ -46,7 +46,7 @@ }, "outputs": "${outputs}" }, - "sparkJobTemplateRef": "DataXDirect", + "sparkJobTemplateRef": "$dataxJobTemplate", "jobCommonTokens": { "jobName": "${name}", "sparkJobName": "DataXDirect-${name}", @@ -54,6 +54,10 @@ "sparkJobExecutorLogLevel": "INFO", "sparkJobNumExecutors": "${guiSparkJobNumExecutors}", "sparkJobExecutorMemory": "${guiSparkJobExecutorMemory}", + "sparkJobDatabricksMinWorkers" : "${guiSparkJobDatabricksMinWorkers}", + "sparkJobDatabricksMaxWorkers" : "${guiSparkJobDatabricksMaxWorkers}", + "sparkDatabricksToken" : "${guiSparkDatabricksToken}", + "sparkJobDatabricksAutoScale" : "${guiSparkJobDatabricksAutoScale}", "processedSchemaPath": null }, "jobs": [ @@ -126,24 +130,25 @@ "cluster": "$sparkName", "options": { "name": "${sparkJobName}", - "file" : "wasbs:///datax/bin/datax-host_2.3_2.11-1.1.0.jar", + "file" : "wasbs:///datax/bin/datax-host_2.4_2.11-1.2.0.jar", "className" : "datax.app.DirectStreamingApp", "args": [ "conf=${sparkJobConfigFilePath}", "driverLogLevel=${sparkJobDriverLogLevel}" ], "jars": [ - "wasbs:///datax/bin/datax-core_2.3_2.11-1.1.0.jar", - "wasbs:///datax/bin/datax-utility_2.3_2.11-1.1.0.jar", + "wasbs:///datax/bin/datax-core_2.4_2.11-1.2.0.jar", + "wasbs:///datax/bin/datax-utility_2.4_2.11-1.2.0.jar", "wasbs:///datax/bin/applicationinsights-core-2.2.1.jar", "wasbs:///datax/bin/azure-documentdb-1.16.1.jar", "wasbs:///datax/bin/azure-eventhubs-1.2.1.jar", "wasbs:///datax/bin/azure-eventhubs-spark_2.11-2.3.6.jar", "wasbs:///datax/bin/azure-keyvault-webkey-1.1.jar", - "wasbs:///datax/bin/datax-keyvault_2.3_2.11-1.1.0-with-dependencies.jar", + "wasbs:///datax/bin/datax-keyvault_2.4_2.11-1.2.0-with-dependencies.jar", "wasbs:///datax/bin/java-uuid-generator-3.1.5.jar", "wasbs:///datax/bin/proton-j-0.31.0.jar", - "wasbs:///datax/bin/scala-java8-compat_2.11-0.9.0.jar" + "wasbs:///datax/bin/scala-java8-compat_2.11-0.9.0.jar", + "wasbs:///datax/bin/azure-sqldb-spark-1.0.2.jar" ], "driverMemory": "1024m", "executorCores": 2, @@ -160,11 +165,17 @@ "spark\uff0especulation": true, "spark\uff0especulation\uff0emultiplier": 3, "spark\uff0especulation\uff0equantile": 0.9, + "spark\uff0eexecutorEnv\uff0eDATAX_AZURESTORAGEJARPATH" : "/datax/bin/azure-storage-3.1.0.jar", + "spark\uff0eexecutorEnv\uff0eDATAX_DEFAULTCONTAINER" : "defaultdx", + "spark\uff0eexecutorEnv\uff0eDATAX_DEFAULTSTORAGEACCOUNT" : "$sparkBlobAccountName", "spark\uff0eexecutorEnv\uff0eDATAX_DEFAULTVAULTNAME" : "$sparkKVName", "spark\uff0eexecutorEnv\uff0eDATAX_APPINSIGHT_KEY": "$appinsightkey", "spark\uff0eexecutorEnv\uff0eDATAX_APPNAME": "${sparkJobName}", "spark\uff0eexecutorEnv\uff0eDATAX_LOGLEVEL": "${sparkJobExecutorLogLevel}", "spark\uff0eexecutorEnv\uff0eDATAX_CHECKPOINTENABLED": "true", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_AZURESTORAGEJARPATH" : "/datax/bin/azure-storage-3.1.0.jar", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_DEFAULTCONTAINER" : "defaultdx", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_DEFAULTSTORAGEACCOUNT" : "$sparkBlobAccountName", "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_DEFAULTVAULTNAME" : "$sparkKVName", "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_APPINSIGHT_KEY": "$appinsightkey", "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_APPNAME": "${sparkJobName}", @@ -174,6 +185,102 @@ } } }, + { + "name": "DataXDirectDatabricks", + "content": { + "name" : "${sparkJobName}", + "cluster" : "$sparkName", + "databricksToken" : "${sparkDatabricksToken}", + "options" : { + "name" : "${sparkJobName}", + "new_cluster" : { + "spark_version" : "$databricksClusterSparkVersion", + "node_type_id" : "$databricksClusterNodeType", + "autoscale" : { + "min_workers" : "${sparkJobDatabricksMinWorkers}", + "max_workers" : "${sparkJobDatabricksMaxWorkers}" + }, + "num_workers" : "${sparkJobDatabricksMinWorkers}", + "enableAutoscale" : "${sparkJobDatabricksAutoScale}", + "spark_conf" : { + "spark\uff0edatabricks\uff0edelta\uff0epreview\uff0eenabled" : true, + "spark\uff0esql\uff0ehive\uff0emetastore\uff0eversion" : "1.2.1", + "spark\uff0edriver\uff0euserClassPathFirst": true, + "spark\uff0eexecutor\uff0euserClassPathFirst": true, + "spark\uff0esql\uff0ehive\uff0emetastore\uff0ejars" : "builtin" + }, + "spark_env_vars" : { + "DATAX_AZURESTORAGEJARPATH" : "/datax/bin/azure-storage-3.1.0.jar", + "DATAX_DEFAULTCONTAINER" : "defaultdx", + "DATAX_DEFAULTSTORAGEACCOUNT" : "$sparkBlobAccountName", + "DATAX_DEFAULTVAULTNAME" : "$sparkKVName" + } + }, + "libraries" : [ + { + "jar" : "dbfs:/datax/applicationinsights-core-2.2.1.jar" + }, + { + "jar" : "dbfs:/datax/azure-documentdb-1.16.1.jar" + }, + { + "jar" : "dbfs:/datax/azure-eventhubs-1.2.1.jar" + }, + { + "jar" : "dbfs:/datax/azure-eventhubs-spark_2.11-2.3.6.jar" + }, + { + "jar" : "dbfs:/datax/azure-keyvault-webkey-1.1.jar" + }, + { + "jar" : "dbfs:/datax/datax-core_2.4_2.11-1.2.0.jar" + }, + { + "jar" : "dbfs:/datax/datax-host_2.4_2.11-1.2.0.jar" + }, + { + "jar" : "dbfs:/datax/datax-keyvault_2.4_2.11-1.2.0-with-dependencies.jar" + }, + { + "jar" : "dbfs:/datax/datax-udf-samples_2.4_2.11-1.2.0.jar" + }, + { + "jar" : "dbfs:/datax/datax-utility_2.4_2.11-1.2.0.jar" + }, + { + "jar" : "dbfs:/datax/java-uuid-generator-3.1.5.jar" + }, + { + "jar" : "dbfs:/datax/proton-j-0.31.0.jar" + }, + { + "jar" : "dbfs:/datax/scala-java8-compat_2.11-0.9.0.jar" + }, + { + "jar" : "dbfs:/datax/azure-sqldb-spark-1.0.2.jar" + }, + { + "jar" : "dbfs:/datax/hadoop-azure-2.7.3.jar" + }, + { + "jar" : "dbfs:/datax/jetty-util-6.1.25.jar" + }, + { + "jar" : "dbfs:/datax/json-20180813.jar" + }, + { + "jar" : "dbfs:/datax/azure-storage-3.1.0.jar" + } + ], + "spark_jar_task" : { + "main_class_name" : "datax.app.DirectStreamingApp", + "parameters" : [ + "conf=${sparkJobConfigFilePath}" + ] + } + } + } + }, { "name" : "flattener", "content" : { @@ -215,6 +322,22 @@ "flushExistingCheckpoints" : "flushexistingcheckpoints" } }, + "blob" : { + "type" : "array", + "namespace" : "blob", + "element" : { + "type" : "scopedObject", + "namespaceField" : "name", + "fields" : { + "path" : "path", + "format" : "format", + "compressiontype" : "compressiontype", + "processstarttime" : "processstarttime", + "processendtime" : "processendtime", + "partitionincrement" : "partitionincrement" + } + } + }, "streaming" : { "type" : "object", "namespace" : "streaming", @@ -348,74 +471,88 @@ "type" : "scopedObject", "namespace" : "output", "namespaceField" : "name", - "fields" : { - "blob" : { - "type" : "object", - "namespace" : "blob", - "fields" : { - "groupEvaluation" : "groupevaluation", - "compressionType" : { - "type" : "excludeDefaultValue", - "namespace" : "compressiontype", - "defaultValue" : "gzip" - }, - "format" : { - "type" : "excludeDefaultValue", - "namespace" : "format", - "defaultValue" : "json" - }, - "groups" : { - "type" : "map", - "namespace" : "group", - "fields" : { - "folder" : "folder" - } - } - } - }, - "eventhub" : { - "type" : "object", - "namespace" : "eventhub", - "fields" : { - "connectionStringRef" : "connectionstring", - "compressionType" : { - "type" : "excludeDefaultValue", - "namespace" : "compressiontype", - "defaultValue" : "gzip" - }, - "format" : { - "type" : "excludeDefaultValue", - "namespace" : "format", - "defaultValue" : "json" - }, - "appendProperties" : { - "type" : "mapProps", - "namespace" : "appendproperty" - } - } - }, - "cosmosdb" : { - "type" : "object", - "namespace" : "cosmosdb", - "fields" : { - "connectionStringRef" : "connectionstring", - "database" : "database", - "collection" : "collection" - } - }, - "httpPost" : { - "type" : "object", - "namespace" : "httppost", - "fields" : { - "endpoint" : "endpoint", - "filter" : "filter", - "appendHeaders" : { - "type" : "mapProps", - "namespace" : "header" - } - } - } - } + "fields": { + "blob": { + "type": "object", + "namespace": "blob", + "fields": { + "groupEvaluation": "groupevaluation", + "compressionType": { + "type": "excludeDefaultValue", + "namespace": "compressiontype", + "defaultValue": "gzip" + }, + "format": { + "type": "excludeDefaultValue", + "namespace": "format", + "defaultValue": "json" + }, + "groups": { + "type": "map", + "namespace": "group", + "fields": { + "folder": "folder" + } + } + } + }, + "eventhub": { + "type": "object", + "namespace": "eventhub", + "fields": { + "connectionStringRef": "connectionstring", + "compressionType": { + "type": "excludeDefaultValue", + "namespace": "compressiontype", + "defaultValue": "gzip" + }, + "format": { + "type": "excludeDefaultValue", + "namespace": "format", + "defaultValue": "json" + }, + "appendProperties": { + "type": "mapProps", + "namespace": "appendproperty" + } + } + }, + "cosmosdb": { + "type": "object", + "namespace": "cosmosdb", + "fields": { + "connectionStringRef": "connectionstring", + "database": "database", + "collection": "collection" + } + }, + "httpPost": { + "type": "object", + "namespace": "httppost", + "fields": { + "endpoint": "endpoint", + "filter": "filter", + "appendHeaders": { + "type": "mapProps", + "namespace": "header" + } + } + }, + "sqlServer": { + "type": "object", + "namespace": "sql", + "fields": { + "connectionStringRef": "connectionstring", + "databaseName": "databasename", + "tableName": "table", + "writeMode": "writemode", + "useBulkInsert": "usebulkinsert", + "url": "url", + "user": "user", + "password": "password" + } + } + } }, "outputs" : { "type" : "array", @@ -423,74 +560,88 @@ "type" : "scopedObject", "namespace" : "output", "namespaceField" : "name", - "fields" : { - "blob" : { - "type" : "object", - "namespace" : "blob", - "fields" : { - "groupEvaluation" : "groupevaluation", - "compressionType" : { - "type" : "excludeDefaultValue", - "namespace" : "compressiontype", - "defaultValue" : "gzip" - }, - "format" : { - "type" : "excludeDefaultValue", - "namespace" : "format", - "defaultValue" : "json" - }, - "groups" : { - "type" : "map", - "namespace" : "group", - "fields" : { - "folder" : "folder" - } - } - } - }, - "eventhub" : { - "type" : "object", - "namespace" : "eventhub", - "fields" : { - "connectionStringRef" : "connectionstring", - "compressionType" : { - "type" : "excludeDefaultValue", - "namespace" : "compressiontype", - "defaultValue" : "gzip" - }, - "format" : { - "type" : "excludeDefaultValue", - "namespace" : "format", - "defaultValue" : "json" - }, - "appendProperties" : { - "type" : "mapProps", - "namespace" : "appendproperty" - } - } - }, - "cosmosdb" : { - "type" : "object", - "namespace" : "cosmosdb", - "fields" : { - "connectionStringRef" : "connectionstring", - "database" : "database", - "collection" : "collection" - } - }, - "httpPost" : { - "type" : "object", - "namespace" : "httppost", - "fields" : { - "endpoint" : "endpoint", - "filter" : "filter", - "appendHeaders" : { - "type" : "mapProps", - "namespace" : "header" - } - } - } - } + "fields": { + "blob": { + "type": "object", + "namespace": "blob", + "fields": { + "groupEvaluation": "groupevaluation", + "compressionType": { + "type": "excludeDefaultValue", + "namespace": "compressiontype", + "defaultValue": "gzip" + }, + "format": { + "type": "excludeDefaultValue", + "namespace": "format", + "defaultValue": "json" + }, + "groups": { + "type": "map", + "namespace": "group", + "fields": { + "folder": "folder" + } + } + } + }, + "eventhub": { + "type": "object", + "namespace": "eventhub", + "fields": { + "connectionStringRef": "connectionstring", + "compressionType": { + "type": "excludeDefaultValue", + "namespace": "compressiontype", + "defaultValue": "gzip" + }, + "format": { + "type": "excludeDefaultValue", + "namespace": "format", + "defaultValue": "json" + }, + "appendProperties": { + "type": "mapProps", + "namespace": "appendproperty" + } + } + }, + "cosmosdb": { + "type": "object", + "namespace": "cosmosdb", + "fields": { + "connectionStringRef": "connectionstring", + "database": "database", + "collection": "collection" + } + }, + "httpPost": { + "type": "object", + "namespace": "httppost", + "fields": { + "endpoint": "endpoint", + "filter": "filter", + "appendHeaders": { + "type": "mapProps", + "namespace": "header" + } + } + }, + "sqlServer": { + "type": "object", + "namespace": "sql", + "fields": { + "connectionStringRef": "connectionstring", + "databaseName": "databasename", + "tableName": "table", + "writeMode": "writemode", + "useBulkInsert": "usebulkinsert", + "url": "url", + "user": "user", + "password": "password" + } + } + } } } } @@ -503,26 +654,27 @@ "cluster": "$sparkName", "options" : { "name" : "${sparkJobName}", - "file" : "wasbs:///datax/bin/datax-host_2.3_2.11-1.1.0.jar", + "file" : "wasbs:///datax/bin/datax-host_2.4_2.11-1.2.0.jar", "className" : "datax.app.DirectKafkaStreamingApp", "args" : [ "conf=${sparkJobConfigFilePath}", "driverLogLevel=${sparkJobDriverLogLevel}" ], "jars" : [ - "wasbs:///datax/bin/datax-core_2.3_2.11-1.1.0.jar", - "wasbs:///datax/bin/datax-utility_2.3_2.11-1.1.0.jar", + "wasbs:///datax/bin/datax-core_2.4_2.11-1.2.0.jar", + "wasbs:///datax/bin/datax-utility_2.4_2.11-1.2.0.jar", "wasbs:///datax/bin/applicationinsights-core-2.2.1.jar", "wasbs:///datax/bin/azure-documentdb-1.16.1.jar", "wasbs:///datax/bin/azure-eventhubs-1.2.1.jar", "wasbs:///datax/bin/azure-eventhubs-spark_2.11-2.3.6.jar", "wasbs:///datax/bin/azure-keyvault-webkey-1.1.jar", - "wasbs:///datax/bin/datax-keyvault_2.3_2.11-1.1.0-with-dependencies.jar", + "wasbs:///datax/bin/datax-keyvault_2.4_2.11-1.2.0-with-dependencies.jar", "wasbs:///datax/bin/java-uuid-generator-3.1.5.jar", "wasbs:///datax/bin/proton-j-0.31.0.jar", "wasbs:///datax/bin/scala-java8-compat_2.11-0.9.0.jar", "wasbs:///datax/bin/spark-streaming-kafka-0-10_2.11-2.4.0.jar", - "wasbs:///datax/bin/kafka-clients-2.0.0.jar" + "wasbs:///datax/bin/kafka-clients-2.0.0.jar", + "wasbs:///datax/bin/azure-sqldb-spark-1.0.2.jar" ], "driverMemory" : "1024m", "executorCores" : 2, @@ -539,11 +691,17 @@ "spark\uff0especulation": true, "spark\uff0especulation\uff0emultiplier": 3, "spark\uff0especulation\uff0equantile": 0.9, + "spark\uff0eexecutorEnv\uff0eDATAX_AZURESTORAGEJARPATH" : "/datax/bin/azure-storage-3.1.0.jar", + "spark\uff0eexecutorEnv\uff0eDATAX_DEFAULTCONTAINER" : "defaultdx", + "spark\uff0eexecutorEnv\uff0eDATAX_DEFAULTSTORAGEACCOUNT" : "$sparkBlobAccountName", "spark\uff0eexecutorEnv\uff0eDATAX_DEFAULTVAULTNAME" : "$sparkKVName", "spark\uff0eexecutorEnv\uff0eDATAX_APPINSIGHT_KEY": "$appinsightkey", "spark\uff0eexecutorEnv\uff0eDATAX_APPNAME": "${sparkJobName}", "spark\uff0eexecutorEnv\uff0eDATAX_LOGLEVEL": "${sparkJobExecutorLogLevel}", "spark\uff0eexecutorEnv\uff0eDATAX_CHECKPOINTENABLED": "true", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_AZURESTORAGEJARPATH" : "/datax/bin/azure-storage-3.1.0.jar", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_DEFAULTCONTAINER" : "defaultdx", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_DEFAULTSTORAGEACCOUNT" : "$sparkBlobAccountName", "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_DEFAULTVAULTNAME" : "$sparkKVName", "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_APPINSIGHT_KEY": "$appinsightkey", "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_APPNAME": "${sparkJobName}", @@ -553,6 +711,108 @@ } } }, + { + "name" : "kafkaDataXDirectDatabricks", + "content" : { + "name" : "${sparkJobName}", + "cluster" : "$sparkName", + "databricksToken" : "${sparkDatabricksToken}", + "options" : { + "name" : "${sparkJobName}", + "new_cluster" : { + "spark_version" : "$databricksClusterSparkVersion", + "node_type_id" : "$databricksClusterNodeType", + "autoscale" : { + "min_workers" : "${sparkJobDatabricksMinWorkers}", + "max_workers" : "${sparkJobDatabricksMaxWorkers}" + }, + "num_workers" : "${sparkJobDatabricksMinWorkers}", + "enableAutoscale" : "${sparkJobDatabricksAutoScale}", + "spark_conf" : { + "spark\uff0edatabricks\uff0edelta\uff0epreview\uff0eenabled" : true, + "spark\uff0esql\uff0ehive\uff0emetastore\uff0eversion" : "1.2.1", + "spark\uff0edriver\uff0euserClassPathFirst": true, + "spark\uff0eexecutor\uff0euserClassPathFirst": true, + "spark\uff0esql\uff0ehive\uff0emetastore\uff0ejars" : "builtin" + }, + "spark_env_vars" : { + "DATAX_AZURESTORAGEJARPATH" : "/datax/bin/azure-storage-3.1.0.jar", + "DATAX_DEFAULTCONTAINER" : "defaultdx", + "DATAX_DEFAULTSTORAGEACCOUNT" : "$sparkBlobAccountName", + "DATAX_DEFAULTVAULTNAME" : "$sparkKVName" + } + }, + "libraries" : [ + { + "jar" : "dbfs:/datax/applicationinsights-core-2.2.1.jar" + }, + { + "jar" : "dbfs:/datax/azure-documentdb-1.16.1.jar" + }, + { + "jar" : "dbfs:/datax/azure-eventhubs-1.2.1.jar" + }, + { + "jar" : "dbfs:/datax/azure-eventhubs-spark_2.11-2.3.6.jar" + }, + { + "jar" : "dbfs:/datax/azure-keyvault-webkey-1.1.jar" + }, + { + "jar" : "dbfs:/datax/datax-core_2.4_2.11-1.2.0.jar" + }, + { + "jar" : "dbfs:/datax/datax-host_2.4_2.11-1.2.0.jar" + }, + { + "jar" : "dbfs:/datax/datax-keyvault_2.4_2.11-1.2.0-with-dependencies.jar" + }, + { + "jar" : "dbfs:/datax/datax-udf-samples_2.4_2.11-1.2.0.jar" + }, + { + "jar" : "dbfs:/datax/datax-utility_2.4_2.11-1.2.0.jar" + }, + { + "jar" : "dbfs:/datax/java-uuid-generator-3.1.5.jar" + }, + { + "jar" : "dbfs:/datax/proton-j-0.31.0.jar" + }, + { + "jar" : "dbfs:/datax/scala-java8-compat_2.11-0.9.0.jar" + }, + { + "jar" : "dbfs:/datax/spark-streaming-kafka-0-10_2.11-2.4.0.jar" + }, + { + "jar" : "dbfs:/datax/kafka-clients-2.0.0.jar" + }, + { + "jar" : "dbfs:/datax/azure-sqldb-spark-1.0.2.jar" + }, + { + "jar" : "dbfs:/datax/hadoop-azure-2.7.3.jar" + }, + { + "jar" : "dbfs:/datax/jetty-util-6.1.25.jar" + }, + { + "jar" : "dbfs:/datax/json-20180813.jar" + }, + { + "jar" : "dbfs:/datax/azure-storage-3.1.0.jar" + } + ], + "spark_jar_task" : { + "main_class_name" : "datax.app.DirectKafkaStreamingApp", + "parameters" : [ + "conf=${sparkJobConfigFilePath}" + ] + } + } + } + }, { "name" : "kafkaFlowConfig", "content" : { @@ -560,7 +820,7 @@ "icon" : "/img/iot.png", "displayName" : "###REQUIRED###", "properties" : { - "inputEventHubSubscriptionId" : "keyvault://${serviceKeyVaultName}/$clientSecretPrefix-subscriptionId", + "inputEventHubSubscriptionId" : "$keyvaultPrefix://${serviceKeyVaultName}/$clientSecretPrefix-subscriptionId", "inputEventHubResourceGroupName" : "${eventHubResourceGroupName}" }, "commonProcessor" : { @@ -586,7 +846,7 @@ }, "process" : { "metric" : { - "eventhub" : "keyvault://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" + "eventhub" : "$keyvaultPrefix://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" }, "timestampColumn" : "${processTimestampColumn}", "watermark" : "${processWatermark}", @@ -601,7 +861,7 @@ }, "outputs" : "${outputs}" }, - "sparkJobTemplateRef" : "kafkaDataXDirect", + "sparkJobTemplateRef" : "$dataxKafkaJobTemplate", "jobCommonTokens" : { "jobName" : "${name}", "sparkJobName" : "DataXDirect-${name}", @@ -609,6 +869,10 @@ "sparkJobExecutorLogLevel" : "INFO", "sparkJobNumExecutors" : "${guiSparkJobNumExecutors}", "sparkJobExecutorMemory" : "${guiSparkJobExecutorMemory}", + "sparkJobDatabricksMinWorkers" : "${guiSparkJobDatabricksMinWorkers}", + "sparkJobDatabricksMaxWorkers" : "${guiSparkJobDatabricksMaxWorkers}", + "sparkDatabricksToken" : "${guiSparkDatabricksToken}", + "sparkJobDatabricksAutoScale" : "${guiSparkJobDatabricksAutoScale}", "processedSchemaPath" : null }, "jobs" : [ @@ -673,5 +937,278 @@ } } } + }, + { + "name" : "batchFlowConfig", + "content" : { + "name" : "###REQUIRED###", + "icon" : "/img/iot.png", + "displayName" : "###REQUIRED###", + "properties" : { + "inputEventHubSubscriptionId" : "$keyvaultPrefix://${serviceKeyVaultName}/web-subscriptionId", + "inputEventHubResourceGroupName" : "${eventHubResourceGroupName}" + }, + "commonProcessor" : { + "sparkJobConfigFolder" : "wasbs://flows@${cpConfigFolderBase}/${containerPath}/${name}", + "sparkJobConfigFolderBackfilling" : "wasbs://flows@${cpConfigFolderBase}/${containerPath}/${name}", + "template" : { + "name" : "${name}", + "input" : { + "blob" : "${inputBatching}", + "blobSchemaFile" : "${inputSchemaFilePath}", + "referenceData" : "${inputReferenceData}" + }, + "process" : { + "metric" : { + "eventhub" : + "$keyvaultPrefix://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" + }, + "timestampColumn" : "${processTimestampColumn}", + "watermark" : "${processWatermark}", + "jarUDAFs" : "${processJarUDAFs}", + "jarUDFs" : "${processJarUDFs}", + "azureFunctions" : "${processAzureFunctions}", + "projections" : "${processProjections}", + "timeWindows" : "${processTimeWindows}", + "transform" : "${processTransforms}", + "appendEventTags" : {}, + "accumulationTables" : "${processStateTables}" + }, + "outputs" : "${outputs}" + }, + "sparkJobTemplateRef" : "$dataxBatchJobTemplate", + "jobCommonTokens" : { + "jobName" : "${name}", + "sparkJobName" : "DataXBatch-${name}", + "sparkJobDriverLogLevel" : "WARN", + "sparkJobExecutorLogLevel" : "INFO", + "sparkJobNumExecutors" : "${guiSparkJobNumExecutors}", + "sparkJobExecutorMemory" : "${guiSparkJobExecutorMemory}", + "sparkJobDatabricksMinWorkers" : "${guiSparkJobDatabricksMinWorkers}", + "sparkJobDatabricksMaxWorkers" : "${guiSparkJobDatabricksMaxWorkers}", + "sparkDatabricksToken" : "${guiSparkDatabricksToken}", + "sparkJobDatabricksAutoScale" : "${guiSparkJobDatabricksAutoScale}", + "processedSchemaPath" : null + }, + "jobs" : [ + { + "partitionJobNumber" : "1" + } + ] + }, + "metrics" : { + "sources" : [ + { + "name" : "events", + "input" : { + "type" : "MetricApi", + "metricKeys" : [ + "DATAX-${name}:Input_DataXProcessedInput_Events_Count" + ] + }, + "output" : { + "type" : "SumWithTimeChart", + "data" : { + "sum" : true, + "timechart" : true, + "average" : true, + "speed" : true + } + } + } + ], + "widgets" : [ + { + "name" : "eventsChart", + "displayName" : "Events/Second", + "data" : "events_timechart", + "position" : "TimeCharts", + "type" : "StackAreaChart" + }, + { + "name" : "totalEvents", + "displayName" : "Events Ingested Today", + "data" : "events_sum", + "formatter" : "longint", + "type" : "SimpleBox", + "position" : "FirstRow" + }, + { + "name" : "averageEvents", + "displayName" : "Avg. Events/Minute", + "data" : "events_average", + "formatter" : "longint", + "type" : "SimpleBox", + "position" : "FirstRow" + } + ], + "initParameters" : { + "widgetSets" : [ + "direct" + ], + "jobNames" : { + "type" : "getCPSparkJobNames" + } + } + } + } + }, + { + "name" : "DataXBatch", + "content" : { + "name" : "${sparkJobName}", + "cluster": "$sparkName", + "options" : { + "name" : "${sparkJobName}", + "file" : "wasbs:///datax/bin/datax-host_2.4_2.11-1.2.0.jar", + "className" : "datax.app.BatchApp", + "args" : [ + "conf=${sparkJobConfigFilePath}", + "driverLogLevel=${sparkJobDriverLogLevel}" + ], + "jars" : [ + "wasbs:///datax/bin/datax-core_2.4_2.11-1.2.0.jar", + "wasbs:///datax/bin/datax-utility_2.4_2.11-1.2.0.jar", + "wasbs:///datax/bin/applicationinsights-core-2.2.1.jar", + "wasbs:///datax/bin/azure-documentdb-1.16.1.jar", + "wasbs:///datax/bin/azure-eventhubs-1.2.1.jar", + "wasbs:///datax/bin/azure-eventhubs-spark_2.11-2.3.6.jar", + "wasbs:///datax/bin/azure-keyvault-webkey-1.1.jar", + "wasbs:///datax/bin/datax-keyvault_2.4_2.11-1.2.0-with-dependencies.jar", + "wasbs:///datax/bin/java-uuid-generator-3.1.5.jar", + "wasbs:///datax/bin/proton-j-0.31.0.jar", + "wasbs:///datax/bin/scala-java8-compat_2.11-0.9.0.jar", + "wasbs:///datax/bin/azure-sqldb-spark-1.0.2.jar" + ], + "driverMemory" : "1024m", + "executorCores" : 2, + "numExecutors" : "${sparkJobNumExecutors}", + "driverCores" : 1, + "executorMemory" : "${sparkJobExecutorMemory}", + "conf" : { + "spark\uff0eexecutor\uff0eextraJavaOptions": "-verbose:gc -XX:+UseG1GC -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintFlagsFinal -XX:+PrintReferenceGC -XX:+PrintAdaptiveSizePolicy -XX:+UnlockDiagnosticVMOptions -XX:+G1SummarizeConcMark -XX:+AlwaysPreTouch -XX:+PrintCodeCache -XX:ReservedCodeCacheSize=240m -XX:+UseCodeCacheFlushing", + "spark\uff0eyarn\uff0eam\uff0eattemptFailuresValidityInterval": "1h", + "spark\uff0eyarn\uff0eexecutor\uff0efailuresValidityInterval": "1h", + "spark\uff0eblacklist\uff0eenabled": "true", + "spark\uff0edriver\uff0euserClassPathFirst": true, + "spark\uff0eexecutor\uff0euserClassPathFirst": true, + "spark\uff0especulation": true, + "spark\uff0especulation\uff0emultiplier": 3, + "spark\uff0especulation\uff0equantile": 0.9, + "spark\uff0eexecutorEnv\uff0eDATAX_AZURESTORAGEJARPATH" : "/datax/bin/azure-storage-3.1.0.jar", + "spark\uff0eexecutorEnv\uff0eDATAX_DEFAULTCONTAINER" : "defaultdx", + "spark\uff0eexecutorEnv\uff0eDATAX_DEFAULTSTORAGEACCOUNT" : "$sparkBlobAccountName", + "spark\uff0eexecutorEnv\uff0eDATAX_DEFAULTVAULTNAME" : "$sparkKVName", + "spark\uff0eexecutorEnv\uff0eDATAX_APPINSIGHT_KEY": "$appinsightkey", + "spark\uff0eexecutorEnv\uff0eDATAX_APPNAME": "${sparkJobName}", + "spark\uff0eexecutorEnv\uff0eDATAX_LOGLEVEL": "${sparkJobExecutorLogLevel}", + "spark\uff0eexecutorEnv\uff0eDATAX_CHECKPOINTENABLED": "true", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_AZURESTORAGEJARPATH" : "/datax/bin/azure-storage-3.1.0.jar", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_DEFAULTCONTAINER" : "defaultdx", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_DEFAULTSTORAGEACCOUNT" : "$sparkBlobAccountName", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_DEFAULTVAULTNAME" : "$sparkKVName", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_APPINSIGHT_KEY": "$appinsightkey", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_APPNAME": "${sparkJobName}", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_LOGLEVEL": "${sparkJobDriverLogLevel}", + "spark\uff0eyarn\uff0eappMasterEnv\uff0eDATAX_CHECKPOINTENABLED": "true" + } + } + } + }, + { + "name" : "DataXBatchDatabricks", + "content" : { + "name" : "${sparkJobName}", + "cluster" : "$sparkName", + "databricksToken" : "${sparkDatabricksToken}", + "options" : { + "name" : "${sparkJobName}", + "new_cluster" : { + "spark_version" : "$databricksClusterSparkVersion", + "node_type_id" : "$databricksClusterNodeType", + "autoscale" : { + "min_workers" : "${sparkJobDatabricksMinWorkers}", + "max_workers" : "${sparkJobDatabricksMaxWorkers}" + }, + "num_workers" : "${sparkJobDatabricksMinWorkers}", + "enableAutoscale" : "${sparkJobDatabricksAutoScale}", + "spark_conf" : { + "spark\uff0edatabricks\uff0edelta\uff0epreview\uff0eenabled" : true, + "spark\uff0esql\uff0ehive\uff0emetastore\uff0eversion" : "1.2.1", + "spark\uff0edriver\uff0euserClassPathFirst": true, + "spark\uff0eexecutor\uff0euserClassPathFirst": true, + "spark\uff0esql\uff0ehive\uff0emetastore\uff0ejars" : "builtin" + }, + "spark_env_vars" : { + "DATAX_AZURESTORAGEJARPATH" : "/datax/bin/azure-storage-3.1.0.jar", + "DATAX_DEFAULTCONTAINER" : "defaultdx", + "DATAX_DEFAULTSTORAGEACCOUNT" : "$sparkBlobAccountName", + "DATAX_DEFAULTVAULTNAME" : "$sparkKVName" + } + }, + "libraries" : [ + { + "jar" : "dbfs:/datax/applicationinsights-core-2.2.1.jar" + }, + { + "jar" : "dbfs:/datax/azure-documentdb-1.16.1.jar" + }, + { + "jar" : "dbfs:/datax/azure-eventhubs-1.2.1.jar" + }, + { + "jar" : "dbfs:/datax/azure-eventhubs-spark_2.11-2.3.6.jar" + }, + { + "jar" : "dbfs:/datax/azure-keyvault-webkey-1.1.jar" + }, + { + "jar" : "dbfs:/datax/datax-core_2.4_2.11-1.2.0.jar" + }, + { + "jar" : "dbfs:/datax/datax-host_2.4_2.11-1.2.0.jar" + }, + { + "jar" : "dbfs:/datax/datax-keyvault_2.4_2.11-1.2.0-with-dependencies.jar" + }, + { + "jar" : "dbfs:/datax/datax-udf-samples_2.4_2.11-1.2.0.jar" + }, + { + "jar" : "dbfs:/datax/datax-utility_2.4_2.11-1.2.0.jar" + }, + { + "jar" : "dbfs:/datax/java-uuid-generator-3.1.5.jar" + }, + { + "jar" : "dbfs:/datax/proton-j-0.31.0.jar" + }, + { + "jar" : "dbfs:/datax/scala-java8-compat_2.11-0.9.0.jar" + }, + { + "jar" : "dbfs:/datax/azure-sqldb-spark-1.0.2.jar" + }, + { + "jar" : "dbfs:/datax/hadoop-azure-2.7.3.jar" + }, + { + "jar" : "dbfs:/datax/jetty-util-6.1.25.jar" + }, + { + "jar" : "dbfs:/datax/json-20180813.jar" + }, + { + "jar" : "dbfs:/datax/azure-storage-3.1.0.jar" + } + ], + "spark_jar_task" : { + "main_class_name" : "datax.app.BatchApp", + "parameters" : [ + "conf=${sparkJobConfigFilePath}" + ] + } + } + } } ] diff --git a/DeploymentCloud/Deployment.Common/CosmosDB/configgenConfigs.json b/DeploymentCloud/Deployment.Common/CosmosDB/configgenConfigs.json index b97411199..6b9928e36 100644 --- a/DeploymentCloud/Deployment.Common/CosmosDB/configgenConfigs.json +++ b/DeploymentCloud/Deployment.Common/CosmosDB/configgenConfigs.json @@ -21,11 +21,13 @@ "sparkKeyVaultName" : "$sparkKVName", "jobURLBase": "$sparkName", "binaryName" : [ - "datax/bin/datax-utility_2.3_2.11-1.1.0.jar", - "datax/bin/datax-host_2.3_2.11-1.1.0.jar", - "datax/bin/datax-core_2.3_2.11-1.1.0.jar" + "datax/bin/datax-utility_2.4_2.11-1.2.0.jar", + "datax/bin/datax-host_2.4_2.11-1.2.0.jar", + "datax/bin/datax-core_2.4_2.11-1.2.0.jar" ], "configgenClientId" : "$configgenClientId", + "configgenClientSecret" : "keyvault://$servicesKVName/$serviceSecretPrefix-clientsecret", + "configgenClientResourceId" : "keyvault://$servicesKVName/$clientSecretPrefix-serviceResourceId", "configgenTenantId" : "$configgenTenantId", "runtimeStorageConnectionString" : "keyvault://$servicesKVName/$serviceSecretPrefix-$sparkBlobAccountName-blobconnectionstring", "metricEventHubConnectionStringKey" : "metric-eventhubconnectionstring", @@ -39,6 +41,8 @@ "interactiveQueryDefaultContainer" : "defaultdx", "sparkClusterName" : "$sparkName", "subscriptionId" : "keyvault://$servicesKVName/$clientSecretPrefix-subscriptionId", - "sparkConnectionString" : "keyvault://$servicesKVName/$serviceSecretPrefix-livyconnectionstring-$sparkName" + "sparkConnectionString" : "keyvault://$servicesKVName/$serviceSecretPrefix-livyconnectionstring-$sparkName", + "sparkRegion" : "$resourceLocation", + "sparkType" : "$sparkType" } ] diff --git a/DeploymentCloud/Deployment.Common/Helpers/UtilityModule.psm1 b/DeploymentCloud/Deployment.Common/Helpers/UtilityModule.psm1 index a855ee7cf..b20609f70 100644 --- a/DeploymentCloud/Deployment.Common/Helpers/UtilityModule.psm1 +++ b/DeploymentCloud/Deployment.Common/Helpers/UtilityModule.psm1 @@ -564,7 +564,7 @@ function Set-AzureAADAccessControl([string]$AppId) { $ErrorActionPreference = "stop" } -function Set-AzureAADApiPermission([string]$ServiceAppId, [string]$ClientAppId) { +function Set-AzureAADApiPermission([string]$ServiceAppId, [string]$ClientAppId, [string]$RoleName) { $ErrorActionPreference = "SilentlyContinue" Write-Host -ForegroundColor Yellow "Setting up App Api Permissions. This requires the subscription admin privilege. If this fails, please refer to the manual steps and ask a subscription admin" @@ -572,6 +572,21 @@ function Set-AzureAADApiPermission([string]$ServiceAppId, [string]$ClientAppId) $aadCommandId = "00000002-0000-0000-c000-000000000000" $permissionId = "311a71cc-e848-46a1-bdf8-97ff7156d8e6" + if ($RoleName) { + $appRoles = az ad app show --id $ServiceAppId --query appRoles | ConvertFrom-Json + + $role = $appRoles | Where-Object { $_.Value -match $RoleName } + if ($role) { + $roleId = $role.Id + az ad app permission add --id $ServiceAppId --api $ServiceAppId --api-permissions $roleId=Role > $null 2>&1 + az ad app permission grant --id $ServiceAppId --api $ServiceAppId --scope $roleId > $null 2>&1 + } + else + { + Write-Host -ForegroundColor Red "$RoleName is not defined in the app $ServiceAppId" + } + } + az ad app permission add --id $ServiceAppId --api $aadCommandId --api-permissions $permissionId=Scope > $null 2>&1 az ad app permission add --id $ClientAppId --api $aadCommandId --api-permissions $permissionId=Scope > $null 2>&1 az ad app permission add --id $ClientAppId --api $ServiceAppId --api-permissions $ServiceAppPermId=Scope > $null 2>&1 diff --git a/DeploymentCloud/Deployment.Common/Resources/Parameters/Databricks-Parameter.json b/DeploymentCloud/Deployment.Common/Resources/Parameters/Databricks-Parameter.json new file mode 100644 index 000000000..38e675eb8 --- /dev/null +++ b/DeploymentCloud/Deployment.Common/Resources/Parameters/Databricks-Parameter.json @@ -0,0 +1,33 @@ +{ + "$schema": "https://schema.management.azure.com/schemas/2015-01-01/deploymentParameters.json#", + "contentVersion": "1.0.0.0", + "parameters": { + "default_resource_location": { + "value": "$resourceLocation" + }, + "sparkClusterName": { + "value": "$sparkName" + }, + "databricksSku": { + "value": "$databricksSku" + }, + "virtualNetworkName": { + "value": "VNet-$serviceFabricName" + }, + "subnetdb1Name": { + "value": "subnet-db-1" + }, + "subnetdb1Prefix": { + "value": "10.0.1.0/24" + }, + "subnetdb2Name": { + "value": "subnet-db-2" + }, + "subnetdb2Prefix": { + "value": "10.0.2.0/24" + }, + "dbResourceGroupName": { + "value": "$dbResourceGroupName" + } + } +} \ No newline at end of file diff --git a/DeploymentCloud/Deployment.Common/Resources/Parameters/Resource-Parameter.json b/DeploymentCloud/Deployment.Common/Resources/Parameters/Resource-Parameter.json index 917bf3abd..5fef24bd6 100644 --- a/DeploymentCloud/Deployment.Common/Resources/Parameters/Resource-Parameter.json +++ b/DeploymentCloud/Deployment.Common/Resources/Parameters/Resource-Parameter.json @@ -32,9 +32,9 @@ "vaults_fabricRDPKV_name": { "value": "$fabricRDPKVName" }, - "sourceVaultValue": { - "value": "$sfKVName" - }, + "sourceVaultValue": { + "value": "$sfKVName" + }, "redis_name": { "value": "$redisName" }, @@ -86,23 +86,29 @@ "certType": { "value": "$certtype" }, - "virtualNetworkName": { - "value": "VNet-$serviceFabricName" - }, - "addressPrefix": { - "value": "10.0.0.0/16" - }, - "computeLocation": { - "value": "$resourceLocationForServiceFabric" - }, - "subnet0Name": { - "value": "Subnet-0" - }, - "subnet0Prefix": { - "value": "10.0.0.0/24" - }, - "clusterName": { - "value": "$serviceFabricName" - } + "virtualNetworkName": { + "value": "VNet-$serviceFabricName" + }, + "addressPrefix": { + "value": "10.0.0.0/16" + }, + "computeLocation": { + "value": "$resourceLocationForServiceFabric" + }, + "subnet0Name": { + "value": "Subnet-0" + }, + "subnet0Prefix": { + "value": "10.0.0.0/24" + }, + "clusterName": { + "value": "$serviceFabricName" + }, + "sparkType": { + "value": "$sparkType" + }, + "storageAccounts_spark_name": { + "value": "$sparkBlobAccountName" + } } } \ No newline at end of file diff --git a/DeploymentCloud/Deployment.Common/Resources/Parameters/Spark-Parameter.json b/DeploymentCloud/Deployment.Common/Resources/Parameters/Spark-Parameter.json index 8991dcf69..99e0bcd95 100644 --- a/DeploymentCloud/Deployment.Common/Resources/Parameters/Spark-Parameter.json +++ b/DeploymentCloud/Deployment.Common/Resources/Parameters/Spark-Parameter.json @@ -52,8 +52,8 @@ "storageAccounts_spark_name": { "value": "$sparkBlobAccountName" }, - "sparkManagedIdentity": { - "value": "$sparkManagedIdentityName" + "sparkManagedIdentity": { + "value": "$sparkManagedIdentityName" }, "minInstanceCountSparkHeadnode": { "value": $minInstanceCountSparkHeadnode @@ -73,11 +73,11 @@ "userAssignedIdentitiesName": { "value": "$sparkManagedIdentityName" }, - "virtualNetworkName": { - "value": "VNet-$serviceFabricName" - }, - "subnet0Name": { - "value": "Subnet-0" - } + "virtualNetworkName": { + "value": "VNet-$serviceFabricName" + }, + "subnet0Name": { + "value": "Subnet-0" + } } } \ No newline at end of file diff --git a/DeploymentCloud/Deployment.Common/Resources/Templates/Databricks-Template.json b/DeploymentCloud/Deployment.Common/Resources/Templates/Databricks-Template.json new file mode 100644 index 000000000..9e84903c3 --- /dev/null +++ b/DeploymentCloud/Deployment.Common/Resources/Templates/Databricks-Template.json @@ -0,0 +1,299 @@ +{ + "$schema": "https://schema.management.azure.com/schemas/2015-01-01/deploymentTemplate.json#", + "contentVersion": "1.0.0.0", + "parameters": { + "default_resource_location": { + "type": "string" + }, + "sparkClusterName": { + "type": "string", + }, + "databricksSku": { + "type": "string", + }, + "virtualNetworkName": { + "type": "string" + }, + "subnetdb1Name": { + "type": "string" + }, + "subnetdb1Prefix": { + "type": "string" + }, + "subnetdb2Name": { + "type": "string" + }, + "subnetdb2Prefix": { + "type": "string" + }, + "dbResourceGroupName": { + "type": "string" + } + }, + "variables": { + "azureRegionToControlPlaneIp": { + "australiacentral": "13.70.105.50/32", + "australiacentral2": "13.70.105.50/32", + "australiaeast": "13.70.105.50/32", + "australiasoutheast": "13.70.105.50/32", + "canadacentral": "40.85.223.25/32", + "canadaeast": "40.85.223.25/32", + "centralindia": "104.211.101.14/32", + "centralus": "23.101.152.95/32", + "eastasia": "52.187.0.85/32", + "eastus": "23.101.152.95/32", + "eastus2": "23.101.152.95/32", + "eastus2euap": "23.101.152.95/32", + "japaneast": "13.78.19.235/32", + "japanwest": "13.78.19.235/32", + "northcentralus": "23.101.152.95/32", + "northeurope": "23.100.0.135/32", + "southcentralus": "40.83.178.242/32", + "southeastasia": "52.187.0.85/32", + "southindia": "104.211.101.14/32", + "uksouth": "51.140.203.27/32", + "ukwest": "51.140.203.27/32", + "westcentralus": "40.83.178.242/32", + "westeurope": "23.100.0.135/32", + "westindia": "104.211.101.14/32", + "westus": "40.83.178.242/32", + "westus2": "40.83.178.242/32", + "koreacentral": "52.141.6.181/32", + "southafricanorth": "40.127.5.82/32" + }, + "azureRegionToWebappIp": { + "australiacentral": "13.75.218.172/32", + "australiacentral2": "13.75.218.172/32", + "australiaeast": "13.75.218.172/32", + "australiasoutheast": "13.75.218.172/32", + "canadacentral": "13.71.184.74/32", + "canadaeast": "13.71.184.74/32", + "centralindia": "104.211.89.81/32", + "centralus": "40.70.58.221/32", + "eastasia": "52.187.145.107/32", + "eastus": "40.70.58.221/32", + "eastus2": "40.70.58.221/32", + "eastus2euap": "40.70.58.221/32", + "japaneast": "52.246.160.72/32", + "japanwest": "52.246.160.72/32", + "northcentralus": "40.70.58.221/32", + "northeurope": "52.232.19.246/32", + "southcentralus": "40.118.174.12/32", + "southeastasia": "52.187.145.107/32", + "southindia": "104.211.89.81/32", + "uksouth": "51.140.204.4/32", + "ukwest": "51.140.204.4/32", + "westcentralus": "40.118.174.12/32", + "westeurope": "52.232.19.246/32", + "westindia": "104.211.89.81/32", + "westus": "40.118.174.12/32", + "westus2": "40.118.174.12/32", + "koreacentral": "52.141.22.164/32", + "southafricanorth": "102.133.224.24/32" + }, + "controlPlaneIp": "[variables('azureRegionToControlPlaneIp')[parameters('default_resource_location')]]", + "webappIp": "[variables('azureRegionToWebappIp')[parameters('default_resource_location')]]", + "nsgName": "[concat(parameters('sparkClusterName'), '-db-nsg')]", + "nsgId": "[resourceId('Microsoft.Network/networkSecurityGroups', variables('nsgName'))]", + "managedResourceGroupName": "[concat('databricks-rg-', parameters('sparkClusterName'), '-', uniqueString(parameters('sparkClusterName'), resourceGroup().id))]" + }, + "resources": [ + { + "apiVersion": "2018-02-01", + "type": "Microsoft.Network/networkSecurityGroups", + "location": "[parameters('default_resource_location')]", + "name": "[variables('nsgName')]", + "properties": { + "securityRules": [ + { + "name": "databricks-worker-to-worker", + "properties": { + "access": "Allow", + "description": "Required for worker nodes communication within a cluster.", + "destinationAddressPrefix": "*", + "destinationPortRange": "*", + "direction": "Inbound", + "priority": 105, + "protocol": "*", + "sourceAddressPrefix": "VirtualNetwork", + "sourcePortRange": "*" + } + }, + { + "name": "databricks-control-plane-ssh", + "properties": { + "access": "Allow", + "description": "Required for Databricks control plane management of worker nodes.", + "destinationAddressPrefix": "*", + "destinationPortRange": "22", + "direction": "Inbound", + "priority": 106, + "protocol": "*", + "sourceAddressPrefix": "[variables('controlPlaneIp')]", + "sourcePortRange": "*" + } + }, + { + "name": "databricks-control-plane-worker-proxy", + "properties": { + "access": "Allow", + "description": "Required for Databricks control plane communication with worker nodes.", + "destinationAddressPrefix": "*", + "destinationPortRange": "5557", + "direction": "Inbound", + "priority": 107, + "protocol": "*", + "sourceAddressPrefix": "[variables('controlPlaneIp')]", + "sourcePortRange": "*" + } + }, + { + "name": "databricks-worker-to-webapp", + "properties": { + "access": "Allow", + "description": "Required for workers communication with Databricks Webapp.", + "destinationAddressPrefix": "[variables('webappIp')]", + "destinationPortRange": "*", + "direction": "Outbound", + "priority": 100, + "protocol": "*", + "sourceAddressPrefix": "*", + "sourcePortRange": "*" + } + }, + { + "name": "databricks-worker-to-sql", + "properties": { + "access": "Allow", + "description": "Required for workers communication with Azure SQL services.", + "destinationAddressPrefix": "Sql", + "destinationPortRange": "*", + "direction": "Outbound", + "priority": 101, + "protocol": "*", + "sourceAddressPrefix": "*", + "sourcePortRange": "*" + } + }, + { + "name": "databricks-worker-to-storage", + "properties": { + "access": "Allow", + "description": "Required for workers communication with Azure Storage services.", + "destinationAddressPrefix": "Storage", + "destinationPortRange": "*", + "direction": "Outbound", + "priority": 102, + "protocol": "*", + "sourceAddressPrefix": "*", + "sourcePortRange": "*" + } + }, + { + "name": "databricks-worker-to-worker-outbound", + "properties": { + "access": "Allow", + "description": "Required for worker nodes communication within a cluster.", + "destinationAddressPrefix": "VirtualNetwork", + "destinationPortRange": "*", + "direction": "Outbound", + "priority": 103, + "protocol": "*", + "sourceAddressPrefix": "*", + "sourcePortRange": "*" + } + }, + { + "name": "databricks-worker-to-any", + "properties": { + "access": "Allow", + "description": "Required for worker nodes communication with any destination.", + "destinationAddressPrefix": "*", + "destinationPortRange": "*", + "direction": "Outbound", + "priority": 104, + "protocol": "*", + "sourceAddressPrefix": "*", + "sourcePortRange": "*" + } + } + ] + } + }, + { + "apiVersion": "2017-05-10", + "name": "nestedTemplate", + "type": "Microsoft.Resources/deployments", + "dependsOn": [ + "[concat('Microsoft.Network/networkSecurityGroups/', concat(parameters('sparkClusterName'), '-db-nsg'))]" + ], + "resourceGroup": "[parameters('dbResourceGroupName')]", + "properties": { + "mode": "Incremental", + "template": { + "$schema": "https://schema.management.azure.com/schemas/2015-01-01/deploymentTemplate.json#", + "contentVersion": "1.0.0.0", + "parameters": {}, + "variables": {}, + "resources": [ + { + "apiVersion": "2018-04-01", + "type": "Microsoft.Network/virtualNetworks/subnets", + "name": "[concat(parameters('virtualNetworkName'), '/', parameters('subnetdb1Name'))]", + "location": "[parameters('default_resource_location')]", + "properties": { + "addressPrefix": "[parameters('subnetdb1Prefix')]", + "networkSecurityGroup": { + "id": "[variables('nsgId')]" + } + } + }, + { + "apiVersion": "2018-04-01", + "type": "Microsoft.Network/virtualNetworks/subnets", + "name": "[concat(parameters('virtualNetworkName'), '/', parameters('subnetdb2Name'))]", + "location": "[parameters('default_resource_location')]", + "dependsOn": [ + "[concat('Microsoft.Network/virtualNetworks/', parameters('virtualNetworkName'), '/subnets/', parameters('subnetdb1Name'))]" + ], + "properties": { + "addressPrefix": "[parameters('subnetdb2Prefix')]", + "networkSecurityGroup": { + "id": "[variables('nsgId')]" + } + } + } + ] + }, + "parameters": {} + } + }, + { + "type": "Microsoft.Databricks/workspaces", + "name": "[parameters('sparkClusterName')]", + "location": "[parameters('default_resource_location')]", + "apiVersion": "2018-04-01", + "dependsOn": [ + "[concat('Microsoft.Network/networkSecurityGroups/', variables('nsgName'))]", + "['Microsoft.Resources/deployments/nestedTemplate']" + ], + "sku": { + "name": "[parameters('databricksSku')]" + }, + "properties": { + "ManagedResourceGroupId": "[concat(subscription().id, '/resourceGroups/', variables('managedResourceGroupName'))]", + "parameters": { + "customVirtualNetworkId": { + "value": "[resourceId('Microsoft.Network/virtualNetworks', parameters('virtualNetworkName'))]" + }, + "customPublicSubnetName": { + "value": "[parameters('subnetdb1Name')]" + }, + "customPrivateSubnetName": { + "value": "[parameters('subnetdb2Name')]" + } + } + } + } + ] +} \ No newline at end of file diff --git a/DeploymentCloud/Deployment.Common/Resources/Templates/Resource-Template.json b/DeploymentCloud/Deployment.Common/Resources/Templates/Resource-Template.json index 96476d1f1..50ad6a295 100644 --- a/DeploymentCloud/Deployment.Common/Resources/Templates/Resource-Template.json +++ b/DeploymentCloud/Deployment.Common/Resources/Templates/Resource-Template.json @@ -32,9 +32,9 @@ "vaults_fabricRDPKV_name": { "type": "String" }, - "sourceVaultValue": { - "type": "string" - }, + "sourceVaultValue": { + "type": "string" + }, "redis_name": { "type": "String" }, @@ -86,35 +86,74 @@ "certType": { "type": "String" }, - "virtualNetworkName": { - "type": "string" - }, - "addressPrefix": { - "type": "string" - }, - "computeLocation": { - "type": "string" - }, - "subnet0Name": { - "type": "string" - }, - "subnet0Prefix": { - "type": "string" - }, - "clusterName": { - "type": "string" - } + "sparkType": { + "type": "String" + }, + "virtualNetworkName": { + "type": "string" + }, + "addressPrefix": { + "type": "string" + }, + "computeLocation": { + "type": "string" + }, + "subnet0Name": { + "type": "string" + }, + "subnet0Prefix": { + "type": "string" + }, + "clusterName": { + "type": "string" + }, + "storageAccounts_spark_name": { + "type": "String" + } }, "variables": { - "vNetApiVersion": "2015-06-15" + "vNetApiVersion": "2015-06-15", + "storageAccounts_containerName_defaultdx": "defaultdx", + "storageAccounts_containerName_diagnostics": "diagnostics", + "storageAccounts_containerName_deployment": "deployment", + "storageAccounts_containerName_scripts": "scripts", + "storageAccounts_containerName_samples": "samples", + "storageAccounts_containerName_usercontent": "usercontent", + "storageAccounts_containerName_flow": "flows", + "storageAccounts_containerName_checkpoints": "checkpoints" }, "resources": [ + { + "type": "Microsoft.Network/networkSecurityGroups", + "apiVersion": "2018-02-01", + "name": "[concat(parameters('virtualNetworkName'),'-nsg')]", + "location": "[parameters('computeLocation')]", + "properties": { + "securityRules": [ + { + "name": "port_443", + "properties": { + "protocol": "*", + "sourcePortRange": "*", + "destinationPortRange": "443", + "sourceAddressPrefix": "*", + "destinationAddressPrefix": "*", + "access": "Allow", + "priority": 100, + "direction": "Inbound" + } + } + ] + } + }, { "apiVersion": "[variables('vNetApiVersion')]", "type": "Microsoft.Network/virtualNetworks", "name": "[parameters('virtualNetworkName')]", "location": "[parameters('computeLocation')]", - "dependsOn": [], + "dependsOn": [ + "[concat('Microsoft.Network/networkSecurityGroups/', concat(parameters('virtualNetworkName'),'-nsg'))]" + ], "properties": { "addressSpace": { "addressPrefixes": [ @@ -125,7 +164,10 @@ { "name": "[parameters('subnet0Name')]", "properties": { - "addressPrefix": "[parameters('subnet0Prefix')]" + "addressPrefix": "[parameters('subnet0Prefix')]", + "networkSecurityGroup" : { + "id": "[resourceId('Microsoft.Network/networkSecurityGroups', concat(parameters('virtualNetworkName'),'-nsg'))]" + } } } ] @@ -598,6 +640,10 @@ { "name": "DATAXDEV_CERTTYPE", "value": "[parameters('certType')]" + }, + { + "name": "DATAX_SPARK_TYPE", + "value": "[parameters('sparkType')]" } ] }, @@ -812,6 +858,81 @@ "dependsOn": [ "[resourceId('Microsoft.Web/sites', parameters('sites_web_name'))]" ] + }, + { + "type": "Microsoft.Storage/storageAccounts", + "name": "[parameters('storageAccounts_spark_name')]", + "apiVersion": "2015-05-01-preview", + "location": "[parameters('default_resource_location')]", + "properties": { + "accountType": "Standard_LRS" + }, + "resources": [ + { + "name": "[concat('default/', variables('storageAccounts_containerName_defaultdx'))]", + "type": "blobServices/containers", + "apiVersion": "2018-03-01-preview", + "dependsOn": [ + "[parameters('storageAccounts_spark_name')]" + ] + }, + { + "name": "[concat('default/', variables('storageAccounts_containerName_diagnostics'))]", + "type": "blobServices/containers", + "apiVersion": "2018-03-01-preview", + "dependsOn": [ + "[parameters('storageAccounts_spark_name')]" + ] + }, + { + "name": "[concat('default/', variables('storageAccounts_containerName_deployment'))]", + "type": "blobServices/containers", + "apiVersion": "2018-03-01-preview", + "dependsOn": [ + "[parameters('storageAccounts_spark_name')]" + ] + }, + { + "name": "[concat('default/', variables('storageAccounts_containerName_scripts'))]", + "type": "blobServices/containers", + "apiVersion": "2018-03-01-preview", + "dependsOn": [ + "[parameters('storageAccounts_spark_name')]" + ] + }, + { + "name": "[concat('default/', variables('storageAccounts_containerName_samples'))]", + "type": "blobServices/containers", + "apiVersion": "2018-03-01-preview", + "dependsOn": [ + "[parameters('storageAccounts_spark_name')]" + ] + }, + { + "name": "[concat('default/', variables('storageAccounts_containerName_usercontent'))]", + "type": "blobServices/containers", + "apiVersion": "2018-03-01-preview", + "dependsOn": [ + "[parameters('storageAccounts_spark_name')]" + ] + }, + { + "name": "[concat('default/', variables('storageAccounts_containerName_flow'))]", + "type": "blobServices/containers", + "apiVersion": "2018-03-01-preview", + "dependsOn": [ + "[parameters('storageAccounts_spark_name')]" + ] + }, + { + "name": "[concat('default/', variables('storageAccounts_containerName_checkpoints'))]", + "type": "blobServices/containers", + "apiVersion": "2018-03-01-preview", + "dependsOn": [ + "[parameters('storageAccounts_spark_name')]" + ] + } + ] } ] } \ No newline at end of file diff --git a/DeploymentCloud/Deployment.Common/Resources/Templates/Spark-Template.json b/DeploymentCloud/Deployment.Common/Resources/Templates/Spark-Template.json index 102f666e5..362befbf4 100644 --- a/DeploymentCloud/Deployment.Common/Resources/Templates/Spark-Template.json +++ b/DeploymentCloud/Deployment.Common/Resources/Templates/Spark-Template.json @@ -55,8 +55,8 @@ "storageAccounts_spark_name": { "type": "String" }, - "sparkManagedIdentity": { - "type": "string" + "sparkManagedIdentity": { + "type": "string" }, "minInstanceCountSparkHeadnode": { "type": "int" @@ -76,42 +76,35 @@ "userAssignedIdentitiesName": { "type": "string" }, - "virtualNetworkName": { - "type": "string" - }, - "subnet0Name": { - "type": "string" - } + "virtualNetworkName": { + "type": "string" + }, + "subnet0Name": { + "type": "string" + } }, "variables": { - "storageAccounts_containerName_scripts": "scripts", - "storageAccounts_containerName_deployment": "deployment", - "storageAccounts_containerName_diagnostics": "diagnostics", - "storageAccounts_containerName_samples": "samples", - "storageAccounts_containerName_usercontent": "usercontent", - "storageAccounts_containerName_flow": "flows", - "storageAccounts_containerName_checkpoints": "checkpoints", "storageAccountsURI": "[concat(parameters('storageAccounts_spark_name'),'.blob.core.windows.net')]", - "storageAccountsResourceID": "[resourceId('Microsoft.Storage/storageAccounts',parameters('storageAccounts_spark_name'))]", + "storageAccountsResourceID": "[resourceId('Microsoft.Storage/storageAccounts',parameters('storageAccounts_spark_name'))]", "userAssignedIdentitiesResourceID": "[resourceId('Microsoft.ManagedIdentity/userAssignedIdentities',parameters('userAssignedIdentitiesName'))]", - "vnetID": "[resourceId('Microsoft.Network/virtualNetworks',parameters('virtualNetworkName'))]", - "subnet0Ref": "[concat(variables('vnetID'),'/subnets/',parameters('subnet0Name'))]" + "vnetID": "[resourceId('Microsoft.Network/virtualNetworks',parameters('virtualNetworkName'))]", + "subnet0Ref": "[concat(variables('vnetID'),'/subnets/',parameters('subnet0Name'))]" }, - "resources": [ - { - "apiVersion": "2015-08-31-preview", - "name": "[parameters('sparkManagedIdentity')]", - "location": "[parameters('default_resource_location')]", - "type": "Microsoft.ManagedIdentity/userAssignedIdentities", - "properties": {} - }, + "resources": [ + { + "apiVersion": "2015-08-31-preview", + "name": "[parameters('sparkManagedIdentity')]", + "location": "[parameters('default_resource_location')]", + "type": "Microsoft.ManagedIdentity/userAssignedIdentities", + "properties": {} + }, { "apiVersion": "2015-03-01-preview", "name": "[parameters('sparkClusterName')]", "type": "Microsoft.HDInsight/clusters", "location": "[parameters('default_resource_location')]", "dependsOn": [ - "[variables('storageAccountsResourceID')]" + "[variables('userAssignedIdentitiesResourceID')]" ], "properties": { "clusterVersion": "[parameters('sparkClusterVersion')]", @@ -190,73 +183,6 @@ "[variables('userAssignedIdentitiesResourceID')]": {} } } - }, - { - "type": "Microsoft.Storage/storageAccounts", - "name": "[parameters('storageAccounts_spark_name')]", - "apiVersion": "2015-05-01-preview", - "location": "[parameters('default_resource_location')]", - "properties": { - "accountType": "Standard_LRS" - }, - "resources": [ - { - "name": "[concat('default/', variables('storageAccounts_containerName_diagnostics'))]", - "type": "blobServices/containers", - "apiVersion": "2018-03-01-preview", - "dependsOn": [ - "[parameters('storageAccounts_spark_name')]" - ] - }, - { - "name": "[concat('default/', variables('storageAccounts_containerName_deployment'))]", - "type": "blobServices/containers", - "apiVersion": "2018-03-01-preview", - "dependsOn": [ - "[parameters('storageAccounts_spark_name')]" - ] - }, - { - "name": "[concat('default/', variables('storageAccounts_containerName_scripts'))]", - "type": "blobServices/containers", - "apiVersion": "2018-03-01-preview", - "dependsOn": [ - "[parameters('storageAccounts_spark_name')]" - ] - }, - { - "name": "[concat('default/', variables('storageAccounts_containerName_samples'))]", - "type": "blobServices/containers", - "apiVersion": "2018-03-01-preview", - "dependsOn": [ - "[parameters('storageAccounts_spark_name')]" - ] - }, - { - "name": "[concat('default/', variables('storageAccounts_containerName_usercontent'))]", - "type": "blobServices/containers", - "apiVersion": "2018-03-01-preview", - "dependsOn": [ - "[parameters('storageAccounts_spark_name')]" - ] - }, - { - "name": "[concat('default/', variables('storageAccounts_containerName_flow'))]", - "type": "blobServices/containers", - "apiVersion": "2018-03-01-preview", - "dependsOn": [ - "[parameters('storageAccounts_spark_name')]" - ] - }, - { - "name": "[concat('default/', variables('storageAccounts_containerName_checkpoints'))]", - "type": "blobServices/containers", - "apiVersion": "2018-03-01-preview", - "dependsOn": [ - "[parameters('storageAccounts_spark_name')]" - ] - } - ] } ] } \ No newline at end of file diff --git a/DeploymentCloud/Deployment.Common/deployResources.ps1 b/DeploymentCloud/Deployment.Common/deployResources.ps1 index 4c7fd6753..a2f5a694d 100644 --- a/DeploymentCloud/Deployment.Common/deployResources.ps1 +++ b/DeploymentCloud/Deployment.Common/deployResources.ps1 @@ -163,6 +163,28 @@ function Get-Tokens { $tokens.Add('tenantId', $tenantId ) $tokens.Add('userId', $userId ) + + $sparkType = 'hdinsight' + $keyvaultPrefix = 'keyvault' + $dataxJobTemplate = 'DataXDirect' + $dataxKafkaJobTemplate = 'kafkaDataXDirect' + $dataxBatchJobTemplate = 'DataXBatch' + if ($useDatabricks -eq 'y') { + $sparkType = 'databricks' + $keyvaultPrefix = 'secretscope' + $dataxJobTemplate = 'DataXDirectDatabricks' + $dataxKafkaJobTemplate = 'kafkaDataXDirectDatabricks' + $dataxBatchJobTemplate = 'DataXBatchDatabricks' + $tokens.Add('databricksClusterSparkVersion', $databricksClusterSparkVersion) + $tokens.Add('databricksClusterNodeType', $databricksClusterNodeType) + $tokens.Add('databricksSku', $databricksSku) + $tokens.Add('dbResourceGroupName', $resourceGroupName) + } + $tokens.Add('sparkType', $sparkType) + $tokens.Add('keyvaultPrefix', $keyvaultPrefix) + $tokens.Add('dataxJobTemplate', $dataxJobTemplate) + $tokens.Add('dataxKafkaJobTemplate', $dataxKafkaJobTemplate) + $tokens.Add('dataxBatchJobTemplate', $dataxBatchJobTemplate) # CosmosDB $tokens.Add('blobopsconnectionString', $blobopsconnectionString ) @@ -227,10 +249,13 @@ function Get-Tokens { } # Get appRole definition -function Create-AppRole([string] $Name, [string] $Description) { +function Create-AppRole([string] $Name, [string] $AppName, [string] $Description) { $appRole = New-Object Microsoft.Open.AzureAD.Model.AppRole $appRole.AllowedMemberTypes = New-Object System.Collections.Generic.List[string] $appRole.AllowedMemberTypes.Add("User"); + if (($Name -eq $writerRole) -and ($AppName -eq $serviceAppName)) { + $appRole.AllowedMemberTypes.Add("Application"); + } $appRole.DisplayName = $Name $appRole.Id = New-Guid $appRole.IsEnabled = $true @@ -241,8 +266,8 @@ function Create-AppRole([string] $Name, [string] $Description) { # Add appRoles to AAD app function Set-AzureAADAppRoles([string]$AppName) { - $role_r = Create-AppRole -Name $readerRole -Description $readerRole + " have ability to view flows" - $role_w = Create-AppRole -Name $writerRole -Description $writerRole + " can manage flows" + $role_r = Create-AppRole -Name $readerRole -AppName $AppName -Description $readerRole + " have ability to view flows" + $role_w = Create-AppRole -Name $writerRole -AppName $AppName -Description $writerRole + " can manage flows" $roles = @($role_r, $role_W) $app = Get-AzureADApplication -Filter "DisplayName eq '$AppName'" @@ -488,6 +513,9 @@ function Setup-SecretsForSpark { $secretName = $prefix + "livyconnectionstring-" + $sparkName $tValue = "endpoint=https://$sparkName.azurehdinsight.net/livy;username=$sparkLogin;password=$sparkPwd" + if ($useDatabricks -eq 'y') { + $tValue = "endpoint=https://$resourceGroupLocation.azuredatabricks.net/api/2.0/;dbtoken=" + } Setup-Secret -VaultName $vaultName -SecretName $secretName -Value $tValue } @@ -615,8 +643,7 @@ function Setup-Secrets { function Setup-KVAccess { # Get ObjectId of web app $servicePrincipalId = az resource show -g $resourceGroupName --name $websiteName --resource-type Microsoft.Web/sites --query identity.principalId - # Get ObjectId of sparkManagedIdentityName - $SparkManagedIdentityId = az resource show -g $resourceGroupName --name $sparkManagedIdentityName --resource-type Microsoft.ManagedIdentity/userAssignedIdentities --query properties.principalId + # Get ObjectId of vmss $vmssId = az resource show -g $resourceGroupName --name $vmNodeTypeName --resource-type Microsoft.Compute/virtualMachineScaleSets --query identity.principalId @@ -630,14 +657,18 @@ function Setup-KVAccess { } az keyvault set-policy --name $servicesKVName --object-id $servicePrincipalId --secret-permissions get, list, set > $null 2>&1 - az keyvault set-policy --name $servicesKVName --object-id $servicePrincipalConfiggenId --secret-permissions get, list, set > $null 2>&1 - az keyvault set-policy --name $servicesKVName --object-id $SparkManagedIdentityId --secret-permissions get, list, set > $null 2>&1 + az keyvault set-policy --name $servicesKVName --object-id $servicePrincipalConfiggenId --secret-permissions get, list, set > $null 2>&1 az keyvault set-policy --name $servicesKVName --object-id $vmssId --secret-permissions get, list, set > $null 2>&1 az keyvault set-policy --name $sparkKVName --object-id $servicePrincipalId --secret-permissions get, list, set > $null 2>&1 az keyvault set-policy --name $sparkKVName --object-id $servicePrincipalConfiggenId --secret-permissions get, list, set, delete > $null 2>&1 - az keyvault set-policy --name $sparkKVName --object-id $SparkManagedIdentityId --secret-permissions get, list, set > $null 2>&1 az keyvault set-policy --name $sparkKVName --object-id $vmssId --secret-permissions get, list, set > $null 2>&1 + if($useDatabricks -eq 'n') { + # Get ObjectId of sparkManagedIdentityName + $SparkManagedIdentityId = az resource show -g $resourceGroupName --name $sparkManagedIdentityName --resource-type Microsoft.ManagedIdentity/userAssignedIdentities --query properties.principalId + az keyvault set-policy --name $servicesKVName --object-id $SparkManagedIdentityId --secret-permissions get, list, set > $null 2>&1 + az keyvault set-policy --name $sparkKVName --object-id $SparkManagedIdentityId --secret-permissions get, list, set > $null 2>&1 + } } # Import SSL Cert To Service Fabric @@ -807,12 +838,18 @@ if($resourceCreation -eq 'y') { } if($sparkCreation -eq 'y') { - Write-Host -ForegroundColor Green "Deploying resources (2/16 steps): A HDInsight cluster will be deployed" - Write-Host -ForegroundColor Green "Estimated time to complete: 20 mins" + Write-Host -ForegroundColor Green "Deploying resources (2/16 steps): A spark cluster will be deployed" Setup-SecretsForSpark $tokens = Get-Tokens - Deploy-Resources -templateName "Spark-Template.json" -paramName "Spark-parameter.json" -templatePath $templatePath -tokens $tokens + if ($useDatabricks -eq 'n') { + Write-Host -ForegroundColor Green "Estimated time to complete: 20 mins" + Deploy-Resources -templateName "Spark-Template.json" -paramName "Spark-parameter.json" -templatePath $templatePath -tokens $tokens + } + else { + Write-Host -ForegroundColor Green "Estimated time to complete: 5 mins" + Deploy-Resources -templateName "Databricks-Template.json" -paramName "Databricks-Parameter.json" -templatePath $templatePath -tokens $tokens + } } # Preparing certs... @@ -850,14 +887,14 @@ Set-AzureAADAppCert -AppName $serviceAppName $azureADAppSecretValue = $azureADAppSecret.Value $azureADAppSecretConfiggenValue = $azureADAppSecretConfiggen.Value -Set-AzureAADAccessControl -AppId $azureADApplicationConfiggenApplicationId -Set-AzureAADApiPermission -ServiceAppId $azureADApplicationConfiggenApplicationId -ClientAppId $azureADApplicationApplicationId - Set-AzureAADAppRoles -AppName $clientAppName Set-AzureAADAppRoles -AppName $serviceAppName Add-UserAppRole -AppName $clientAppName Add-UserAppRole -AppName $serviceAppName +Set-AzureAADAccessControl -AppId $azureADApplicationConfiggenApplicationId +Set-AzureAADApiPermission -ServiceAppId $azureADApplicationConfiggenApplicationId -ClientAppId $azureADApplicationApplicationId -RoleName $writerRole + if($serviceFabricCreation -eq 'y') { Write-Host -ForegroundColor Green "Deploying resources (4/16 steps): A Service fabric cluster will be deployed" Write-Host -ForegroundColor Green "Estimated time to complete: 20 mins" @@ -890,9 +927,11 @@ if ($setupSecrets -eq 'y') { # Spark if ($sparkCreation -eq 'y') { - Write-Host -ForegroundColor Green "Setting up ScriptActions... (6/16 steps)" - Write-Host -ForegroundColor Green "Estimated time to complete: 2 mins" - Add-ScriptActions + Write-Host -ForegroundColor Green "Setting up ScriptActions... (6/16 steps)" + if ($useDatabricks -eq 'n') { + Write-Host -ForegroundColor Green "Estimated time to complete: 2 mins" + Add-ScriptActions + } } # cosmosDB diff --git a/DeploymentCloud/Deployment.DataX/Resources/Parameters/IotHub-Parameter.json b/DeploymentCloud/Deployment.DataX/Resources/Parameters/IotHub-Parameter.json index afb123810..1947f24fd 100644 --- a/DeploymentCloud/Deployment.DataX/Resources/Parameters/IotHub-Parameter.json +++ b/DeploymentCloud/Deployment.DataX/Resources/Parameters/IotHub-Parameter.json @@ -9,7 +9,7 @@ "value": "$resourceLocation" }, "sku_name": { - "value": "B3" + "value": "B1" }, "sku_units": { "value": "1" diff --git a/DeploymentCloud/Deployment.DataX/Resources/Templates/Kafka-Template.json b/DeploymentCloud/Deployment.DataX/Resources/Templates/Kafka-Template.json index fb74f5a11..28be9eed2 100644 --- a/DeploymentCloud/Deployment.DataX/Resources/Templates/Kafka-Template.json +++ b/DeploymentCloud/Deployment.DataX/Resources/Templates/Kafka-Template.json @@ -136,7 +136,7 @@ "name": "zookeepernode", "targetInstanceCount": 3, "hardwareProfile": { - "vmSize": "A5" + "vmSize": "Standard_A4_V2" }, "osProfile": { "linuxOperatingSystemProfile": { diff --git a/DeploymentCloud/Deployment.DataX/Samples/flows/eventhub-product.json b/DeploymentCloud/Deployment.DataX/Samples/flows/eventhub-product.json index 9ed63d001..8fd0baf80 100644 --- a/DeploymentCloud/Deployment.DataX/Samples/flows/eventhub-product.json +++ b/DeploymentCloud/Deployment.DataX/Samples/flows/eventhub-product.json @@ -24,7 +24,7 @@ }, "process" : { "metric" : { - "eventhub" : "keyvault://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" + "eventhub" : "$keyvaultPrefix://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" }, "timestampColumn" : "${processTimestampColumn}", "watermark" : "${processWatermark}", @@ -48,6 +48,10 @@ "sparkJobExecutorLogLevel" : "INFO", "sparkJobNumExecutors" : "${guiSparkJobNumExecutors}", "sparkJobExecutorMemory" : "${guiSparkJobExecutorMemory}", + "sparkJobDatabricksMinWorkers" : "${guiSparkJobDatabricksMinWorkers}", + "sparkJobDatabricksMaxWorkers" : "${guiSparkJobDatabricksMaxWorkers}", + "sparkDatabricksToken" : "${guiSparkDatabricksToken}", + "sparkJobDatabricksAutoScale" : "${guiSparkJobDatabricksAutoScale}", "processedSchemaPath" : null }, "jobs" : [ @@ -362,13 +366,14 @@ "gui" : { "name" : "eventhub", "displayName" : "EventHub", + "databricksToken" : "", "owner" : "eventhub", "input" : { "mode" : "streaming", "type" : "events", "properties" : { "inputEventhubName" : "", - "inputEventhubConnection" : "keyvault://$sparkKVName/eventhub-input-eventhubconnectionstring", + "inputEventhubConnection" : "$keyvaultPrefix://$sparkKVName/eventhub-input-eventhubconnectionstring", "windowDuration" : "60", "timestampColumn" : "eventTimeStamp", "watermarkValue" : "60", @@ -384,7 +389,7 @@ "type" : "csv", "typeDisplay" : "CSV/TSV File", "properties" : { - "path" : "keyvault://$sparkKVName/eventhub-referencedata-devicesdata", + "path" : "$keyvaultPrefix://$sparkKVName/eventhub-referencedata-devicesdata", "delimiter" : ",", "header" : true } @@ -400,7 +405,7 @@ "type" : "jarUDF", "properties" : { "name" : null, - "path" : "keyvault://$sparkKVName/eventhub-jarpath-udfsample", + "path" : "$keyvaultPrefix://$sparkKVName/eventhub-jarpath-udfsample", "class" : "datax.sample.udf.UdfHelloWorld", "libs" : [ ] }, @@ -412,7 +417,10 @@ ], "jobconfig" : { "jobNumExecutors" : "4", - "jobExecutorMemory" : "1024" + "jobExecutorMemory" : "1024", + "jobDatabricksAutoScale" : false, + "jobDatabricksMinWorkers" : "3", + "jobDatabricksMaxWorkers" : "8" } }, "outputs" : [ @@ -526,10 +534,8 @@ "subscription" : "" }, "properties" : { - "inputEventHubSubscriptionId" : "keyvault://${serviceKeyVaultName}/$clientSecretPrefix-subscriptionId", + "inputEventHubSubscriptionId" : "$keyvaultPrefix://${serviceKeyVaultName}/$clientSecretPrefix-subscriptionId", "inputEventHubResourceGroupName" : "${eventHubResourceGroupName}" }, - "jobNames" : [ - "DataXDirect-eventhub" - ] + "jobNames" : null } \ No newline at end of file diff --git a/DeploymentCloud/Deployment.DataX/Samples/flows/eventhubkafka-product.json b/DeploymentCloud/Deployment.DataX/Samples/flows/eventhubkafka-product.json index 91691a445..da58bbf47 100644 --- a/DeploymentCloud/Deployment.DataX/Samples/flows/eventhubkafka-product.json +++ b/DeploymentCloud/Deployment.DataX/Samples/flows/eventhubkafka-product.json @@ -25,7 +25,7 @@ }, "process" : { "metric" : { - "eventhub" : "keyvault://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" + "eventhub" : "$keyvaultPrefix://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" }, "timestampColumn" : "${processTimestampColumn}", "watermark" : "${processWatermark}", @@ -49,6 +49,10 @@ "sparkJobExecutorLogLevel" : "INFO", "sparkJobNumExecutors" : "${guiSparkJobNumExecutors}", "sparkJobExecutorMemory" : "${guiSparkJobExecutorMemory}", + "sparkJobDatabricksMinWorkers" : "${guiSparkJobDatabricksMinWorkers}", + "sparkJobDatabricksMaxWorkers" : "${guiSparkJobDatabricksMaxWorkers}", + "sparkDatabricksToken" : "${guiSparkDatabricksToken}", + "sparkJobDatabricksAutoScale" : "${guiSparkJobDatabricksAutoScale}", "processedSchemaPath" : null }, "jobs" : [ @@ -363,13 +367,14 @@ "gui" : { "name" : "eventhubkafka", "displayName" : "EventhubKafka", + "databricksToken" : "", "owner" : "eventhubkafka", "input" : { "mode" : "streaming", "type" : "kafkaeventhub", "properties" : { "inputEventhubName" : "kafka1,kafka2", - "inputEventhubConnection" : "keyvault://$sparkKVName/eventhubkafka-input-eventhubconnectionstring", + "inputEventhubConnection" : "$keyvaultPrefix://$sparkKVName/eventhubkafka-input-eventhubconnectionstring", "windowDuration" : "60", "timestampColumn" : "eventTimeStamp", "watermarkValue" : "60", @@ -385,7 +390,7 @@ "type" : "csv", "typeDisplay" : "CSV/TSV File", "properties" : { - "path" : "keyvault://$sparkKVName/eventhubkafka-referencedata-devicesdata", + "path" : "$keyvaultPrefix://$sparkKVName/eventhubkafka-referencedata-devicesdata", "delimiter" : ",", "header" : true } @@ -401,7 +406,7 @@ "type" : "jarUDF", "properties" : { "name" : null, - "path" : "keyvault://$sparkKVName/eventhubkafka-jarpath-udfsample", + "path" : "$keyvaultPrefix://$sparkKVName/eventhubkafka-jarpath-udfsample", "class" : "datax.sample.udf.UdfHelloWorld", "libs" : [ ] }, @@ -413,7 +418,10 @@ ], "jobconfig" : { "jobNumExecutors" : "4", - "jobExecutorMemory" : "1024" + "jobExecutorMemory" : "1024", + "jobDatabricksAutoScale" : false, + "jobDatabricksMinWorkers" : "3", + "jobDatabricksMaxWorkers" : "8" } }, "outputs" : [ @@ -527,10 +535,8 @@ "subscription" : "" }, "properties" : { - "inputEventHubSubscriptionId" : "keyvault://${serviceKeyVaultName}/$clientSecretPrefix-subscriptionId", + "inputEventHubSubscriptionId" : "$keyvaultPrefix://${serviceKeyVaultName}/$clientSecretPrefix-subscriptionId", "inputEventHubResourceGroupName" : "${eventHubResourceGroupName}" }, - "jobNames" : [ - "DataXDirect-eventhubkafka" - ] + "jobNames" : null } \ No newline at end of file diff --git a/DeploymentCloud/Deployment.DataX/Samples/flows/iotsample-product.json b/DeploymentCloud/Deployment.DataX/Samples/flows/iotsample-product.json index 668e571a0..62b53fc79 100644 --- a/DeploymentCloud/Deployment.DataX/Samples/flows/iotsample-product.json +++ b/DeploymentCloud/Deployment.DataX/Samples/flows/iotsample-product.json @@ -24,7 +24,7 @@ }, "process" : { "metric" : { - "eventhub" : "keyvault://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" + "eventhub" : "$keyvaultPrefix://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" }, "timestampColumn" : "${processTimestampColumn}", "watermark" : "${processWatermark}", @@ -48,6 +48,10 @@ "sparkJobExecutorLogLevel" : "INFO", "sparkJobNumExecutors" : "${guiSparkJobNumExecutors}", "sparkJobExecutorMemory" : "${guiSparkJobExecutorMemory}", + "sparkJobDatabricksMinWorkers" : "${guiSparkJobDatabricksMinWorkers}", + "sparkJobDatabricksMaxWorkers" : "${guiSparkJobDatabricksMaxWorkers}", + "sparkDatabricksToken" : "${guiSparkDatabricksToken}", + "sparkJobDatabricksAutoScale" : "${guiSparkJobDatabricksAutoScale}", "processedSchemaPath" : null }, "jobs" : [ @@ -362,13 +366,14 @@ "gui" : { "name" : "iotsample", "displayName" : "IoT Sample", + "databricksToken" : "", "owner" : "iotsample", "input" : { "mode" : "streaming", "type" : "iothub", "properties" : { "inputEventhubName" : "$iotHubName", - "inputEventhubConnection" : "keyvault://$sparkKVName/iotsample-input-eventhubconnectionstring", + "inputEventhubConnection" : "$keyvaultPrefix://$sparkKVName/iotsample-input-eventhubconnectionstring", "windowDuration" : "60", "timestampColumn" : "eventTimeStamp", "watermarkValue" : "60", @@ -384,7 +389,7 @@ "type" : "csv", "typeDisplay" : "CSV/TSV File", "properties" : { - "path" : "keyvault://$sparkKVName/iotsample-referencedata-devicesdata", + "path" : "$keyvaultPrefix://$sparkKVName/iotsample-referencedata-devicesdata", "delimiter" : ",", "header" : true } @@ -400,7 +405,7 @@ "type" : "jarUDF", "properties" : { "name" : null, - "path" : "keyvault://$sparkKVName/iotsample-jarpath-udfsample", + "path" : "$keyvaultPrefix://$sparkKVName/iotsample-jarpath-udfsample", "class" : "datax.sample.udf.UdfHelloWorld", "libs" : [ ] }, @@ -412,7 +417,10 @@ ], "jobconfig" : { "jobNumExecutors" : "4", - "jobExecutorMemory" : "1024" + "jobExecutorMemory" : "1024", + "jobDatabricksAutoScale" : false, + "jobDatabricksMinWorkers" : "3", + "jobDatabricksMaxWorkers" : "8" } }, "outputs" : [ @@ -526,10 +534,8 @@ "subscription" : "" }, "properties" : { - "inputEventHubSubscriptionId" : "keyvault://${serviceKeyVaultName}/$clientSecretPrefix-subscriptionId", + "inputEventHubSubscriptionId" : "$keyvaultPrefix://${serviceKeyVaultName}/$clientSecretPrefix-subscriptionId", "inputEventHubResourceGroupName" : "${eventHubResourceGroupName}" }, - "jobNames" : [ - "DataXDirect-iotsample" - ] + "jobNames" : null } \ No newline at end of file diff --git a/DeploymentCloud/Deployment.DataX/Samples/flows/nativekafka-product.json b/DeploymentCloud/Deployment.DataX/Samples/flows/nativekafka-product.json index c71009666..12778bb8d 100644 --- a/DeploymentCloud/Deployment.DataX/Samples/flows/nativekafka-product.json +++ b/DeploymentCloud/Deployment.DataX/Samples/flows/nativekafka-product.json @@ -25,7 +25,7 @@ }, "process" : { "metric" : { - "eventhub" : "keyvault://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" + "eventhub" : "$keyvaultPrefix://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" }, "timestampColumn" : "${processTimestampColumn}", "watermark" : "${processWatermark}", @@ -49,6 +49,10 @@ "sparkJobExecutorLogLevel" : "INFO", "sparkJobNumExecutors" : "${guiSparkJobNumExecutors}", "sparkJobExecutorMemory" : "${guiSparkJobExecutorMemory}", + "sparkJobDatabricksMinWorkers" : "${guiSparkJobDatabricksMinWorkers}", + "sparkJobDatabricksMaxWorkers" : "${guiSparkJobDatabricksMaxWorkers}", + "sparkDatabricksToken" : "${guiSparkDatabricksToken}", + "sparkJobDatabricksAutoScale" : "${guiSparkJobDatabricksAutoScale}", "processedSchemaPath" : null }, "jobs" : [ @@ -363,13 +367,14 @@ "gui" : { "name" : "nativekafka", "displayName" : "NativeKafka", + "databricksToken" : "", "owner" : "nativekafka", "input" : { "mode" : "streaming", "type" : "kafka", "properties" : { "inputEventhubName" : "kafka1,kafka2", - "inputEventhubConnection" : "keyvault://$sparkKVName/nativekafka-input-eventhubconnectionstring", + "inputEventhubConnection" : "$keyvaultPrefix://$sparkKVName/nativekafka-input-eventhubconnectionstring", "windowDuration" : "60", "timestampColumn" : "eventTimeStamp", "watermarkValue" : "60", @@ -385,7 +390,7 @@ "type" : "csv", "typeDisplay" : "CSV/TSV File", "properties" : { - "path" : "keyvault://$sparkKVName/nativekafka-referencedata-devicesdata", + "path" : "$keyvaultPrefix://$sparkKVName/nativekafka-referencedata-devicesdata", "delimiter" : ",", "header" : true } @@ -401,7 +406,7 @@ "type" : "jarUDF", "properties" : { "name" : null, - "path" : "keyvault://$sparkKVName/nativekafka-jarpath-udfsample", + "path" : "$keyvaultPrefix://$sparkKVName/nativekafka-jarpath-udfsample", "class" : "datax.sample.udf.UdfHelloWorld", "libs" : [ ] }, @@ -413,7 +418,10 @@ ], "jobconfig" : { "jobNumExecutors" : "4", - "jobExecutorMemory" : "1024" + "jobExecutorMemory" : "1024", + "jobDatabricksAutoScale" : false, + "jobDatabricksMinWorkers" : "3", + "jobDatabricksMaxWorkers" : "8" } }, "outputs" : [ @@ -527,10 +535,8 @@ "subscription" : "" }, "properties" : { - "inputEventHubSubscriptionId" : "keyvault://${serviceKeyVaultName}/$clientSecretPrefix-subscriptionId", + "inputEventHubSubscriptionId" : "$keyvaultPrefix://${serviceKeyVaultName}/$clientSecretPrefix-subscriptionId", "inputEventHubResourceGroupName" : "${eventHubResourceGroupName}" }, - "jobNames" : [ - "DataXDirect-nativekafka" - ] + "jobNames" : null } \ No newline at end of file diff --git a/DeploymentCloud/Deployment.DataX/Samples/samples/iotDevice/iotsample.json b/DeploymentCloud/Deployment.DataX/Samples/samples/iotDevice/iotsample.json index e7a755f37..cfbcee551 100644 --- a/DeploymentCloud/Deployment.DataX/Samples/samples/iotDevice/iotsample.json +++ b/DeploymentCloud/Deployment.DataX/Samples/samples/iotDevice/iotsample.json @@ -4,7 +4,7 @@ { "dataTypeName": "DeviceData", "simulationPeriodInMinute": 1, - "numEventsPerBatch": 100, + "numEventsPerBatch": 50, "fields": [ { "name": "deviceDetails", diff --git a/DeploymentCloud/Deployment.DataX/Services/Parameters/DataXFlow.Parameters.json b/DeploymentCloud/Deployment.DataX/Services/Parameters/DataXFlow.Parameters.json index 28832adbc..d7091509e 100644 --- a/DeploymentCloud/Deployment.DataX/Services/Parameters/DataXFlow.Parameters.json +++ b/DeploymentCloud/Deployment.DataX/Services/Parameters/DataXFlow.Parameters.json @@ -24,7 +24,8 @@ "cosmosDBConfigDatabaseName": "keyvault://$servicesKVName/$serviceSecretPrefix-configgenconfigsdatabasename", "cosmosDBConfigCollectionName": "configgenConfigs", "AppInsightsIntrumentationKey": "$serviceSecretPrefix-aiInstrumentationKey", - "CACertificateLocation": "keyvault://$servicesKVName/$serviceSecretPrefix-cacertificatelocation" + "CACertificateLocation": "keyvault://$servicesKVName/$serviceSecretPrefix-cacertificatelocation", + "SparkType": "$sparkType" } } } diff --git a/DeploymentCloud/Deployment.DataX/adminSteps.ps1 b/DeploymentCloud/Deployment.DataX/adminSteps.ps1 index cdcde6dab..69ae38001 100644 --- a/DeploymentCloud/Deployment.DataX/adminSteps.ps1 +++ b/DeploymentCloud/Deployment.DataX/adminSteps.ps1 @@ -41,6 +41,6 @@ Write-Host "Signing in '$tenantId'" az login --tenant $tenantId Set-AzureAADAccessControl -AppId $serviceAppId -Set-AzureAADApiPermission -ServiceAppId $serviceAppId -ClientAppId $clientAppId +Set-AzureAADApiPermission -ServiceAppId $serviceAppId -ClientAppId $clientAppId -RoleName $writerRole Exit 0 \ No newline at end of file diff --git a/DeploymentCloud/Deployment.DataX/common.parameters.txt b/DeploymentCloud/Deployment.DataX/common.parameters.txt index 108aedfa5..35e8176f6 100644 --- a/DeploymentCloud/Deployment.DataX/common.parameters.txt +++ b/DeploymentCloud/Deployment.DataX/common.parameters.txt @@ -20,6 +20,18 @@ subscriptionId= # ResourceGroupName to generate resources resourceGroupName=DataX +# y if you want to run spark jobs on Databricks, n for running jobs on HDinsight +useDatabricks=n + +#Databricks cluster spark version +databricksClusterSparkVersion=5.3.x-scala2.11 + +#Databricks cluster node type id +databricksClusterNodeType=Standard_DS3_v2 + +#Databricks SKU +databricksSku=premium + # y if you want to deploy Kafka sample, n otherwise # This will deploy the Kafka specific samples and resources such as HDinsight Kafka and EventHub Kafka enableKafkaSample=y diff --git a/DeploymentCloud/Deployment.DataX/deploySample.ps1 b/DeploymentCloud/Deployment.DataX/deploySample.ps1 index 190a2c085..600b784cf 100644 --- a/DeploymentCloud/Deployment.DataX/deploySample.ps1 +++ b/DeploymentCloud/Deployment.DataX/deploySample.ps1 @@ -22,7 +22,7 @@ if ($deploySample -ne 'y') { } $rootFolderPath = $PSScriptRoot -Import-Module "..\Deployment.Common\Helpers\UtilityModule" -ArgumentList $rootFolderPath, $resourceGroupName, $productName, $sparkClusterName -WarningAction SilentlyContinue +Import-Module "..\Deployment.Common\Helpers\UtilityModule" -ArgumentList $rootFolderPath, $resourceGroupName, $productName, $sparkClusterName, $randomizeProductName, $serviceFabricClusterName -WarningAction SilentlyContinue # Check if file paths exist function Check-FilePath { @@ -185,6 +185,12 @@ function Get-Tokens { $tokens.Add('kafkaName', $kafkaName) $tokens.Add('kafkaEventHubNamespaceName', $kafkaEventHubNamespaceName) $tokens.Add('clientSecretPrefix', $clientSecretPrefix) + + $keyvaultPrefix = 'keyvault' + if ($useDatabricks -eq 'y') { + $keyvaultPrefix = 'secretscope' + } + $tokens.Add('keyvaultPrefix', $keyvaultPrefix) $tokens } diff --git a/DeploymentCloud/Deployment.Kubernetes/Flow.ManagementService/Dockerfile b/DeploymentCloud/Deployment.Kubernetes/Dockerfile similarity index 67% rename from DeploymentCloud/Deployment.Kubernetes/Flow.ManagementService/Dockerfile rename to DeploymentCloud/Deployment.Kubernetes/Dockerfile index a43b75144..145c6938b 100644 --- a/DeploymentCloud/Deployment.Kubernetes/Flow.ManagementService/Dockerfile +++ b/DeploymentCloud/Deployment.Kubernetes/Dockerfile @@ -1,9 +1,12 @@ FROM microsoft/dotnet:2.2-aspnetcore-runtime WORKDIR /app/aspnetcore EXPOSE 5000 -ADD "FlowManagement.tar" . +ARG servicetarname +ARG servicedllname +ENV servicedllname=$servicedllname +ADD $servicetarname . ENV ASPNETCORE_URLS="http://*:5000" PATH="$PATH:/app/aspnetcore:/app/aspnetcore/" COPY "finalrun.sh" "./" # Adding the execute permission. RUN chmod +x ./finalrun.sh -ENTRYPOINT ["finalrun.sh"] \ No newline at end of file +ENTRYPOINT ["finalrun.sh"] diff --git a/DeploymentCloud/Deployment.Kubernetes/Flow.ManagementService/finalrun.sh b/DeploymentCloud/Deployment.Kubernetes/Flow.ManagementService/finalrun.sh deleted file mode 100644 index 4a5e37f7b..000000000 --- a/DeploymentCloud/Deployment.Kubernetes/Flow.ManagementService/finalrun.sh +++ /dev/null @@ -1,2 +0,0 @@ -#!/bin/bash -dotnet Flow.ManagementService.dll \ No newline at end of file diff --git a/DeploymentCloud/Deployment.Kubernetes/finalrun.sh b/DeploymentCloud/Deployment.Kubernetes/finalrun.sh new file mode 100644 index 000000000..f900f96b0 --- /dev/null +++ b/DeploymentCloud/Deployment.Kubernetes/finalrun.sh @@ -0,0 +1,2 @@ +#!/bin/bash +dotnet "$servicedllname" diff --git a/DeploymentCloud/Deployment.Kubernetes/flow-interactive-query-service.yaml b/DeploymentCloud/Deployment.Kubernetes/flow-interactive-query-service.yaml new file mode 100644 index 000000000..922c730a6 --- /dev/null +++ b/DeploymentCloud/Deployment.Kubernetes/flow-interactive-query-service.yaml @@ -0,0 +1,64 @@ +--- +apiVersion: apps/v1beta1 +kind: Deployment +metadata: + name: flow-interactive-query-service +spec: + replicas: 1 + strategy: + rollingUpdate: + maxSurge: 1 + maxUnavailable: 1 + minReadySeconds: 5 + template: + metadata: + labels: + app: flow-interactive-query-service + aadpodidbinding: managed-service-identity + spec: + terminationGracePeriodSeconds: 30 + nodeSelector: + "beta.kubernetes.io/os": linux + containers: + - name: flow-interactive-query-service + image: dataxregistry.azurecr.io/datax/cloud/flow.interactivequeryservice:v1 + imagePullPolicy: "Always" + ports: + - containerPort: 5000 + resources: + requests: + cpu: 250m + limits: + cpu: 500m + env: + - name: ASPNETCORE_URLS + value: "http://+:5000" + - name: DataX__ServiceEnvironment__AppInsightsIntrumentationKey + value: "" + - name: DataX__ServiceEnvironment__CosmosDBConfigCollectionName + value: "" + - name: DataX__ServiceEnvironment__CosmosDBConfigConnectionString + value: "" + - name: DataX__ServiceEnvironment__CosmosDBConfigDatabaseName + value: "" + - name: DataX__ServiceEnvironment__ServiceKeyVaultName + value: "" + - name: JwtBearerOptions__Audience + value: "" + - name: JwtBearerOptions__Authority + value: "" + - name: servicedllname + value: "Flow.InteractiveQueryService.dll" + - name: DataX__ServiceEnvironment__EnableOneBox + value: "false" +--- +apiVersion: v1 +kind: Service +metadata: + name: flow-interactive-query-service +spec: + type: LoadBalancer + ports: + - port: 5000 + selector: + app: flow-interactive-query-service diff --git a/DeploymentCloud/Deployment.Kubernetes/flow-live-data-service.yaml b/DeploymentCloud/Deployment.Kubernetes/flow-live-data-service.yaml new file mode 100644 index 000000000..5bfbda3ff --- /dev/null +++ b/DeploymentCloud/Deployment.Kubernetes/flow-live-data-service.yaml @@ -0,0 +1,64 @@ +--- +apiVersion: apps/v1beta1 +kind: Deployment +metadata: + name: flow-live-data-service +spec: + replicas: 1 + strategy: + rollingUpdate: + maxSurge: 1 + maxUnavailable: 1 + minReadySeconds: 5 + template: + metadata: + labels: + app: flow-live-data-service + aadpodidbinding: managed-service-identity + spec: + terminationGracePeriodSeconds: 30 + nodeSelector: + "beta.kubernetes.io/os": linux + containers: + - name: flow-live-data-service + image: dataxregistry.azurecr.io/datax/cloud/flow.livedataservice:v1 + imagePullPolicy: "Always" + ports: + - containerPort: 5000 + resources: + requests: + cpu: 250m + limits: + cpu: 500m + env: + - name: ASPNETCORE_URLS + value: "http://+:5000" + - name: DataX__ServiceEnvironment__AppInsightsIntrumentationKey + value: "" + - name: DataX__ServiceEnvironment__CosmosDBConfigCollectionName + value: "" + - name: DataX__ServiceEnvironment__CosmosDBConfigConnectionString + value: "" + - name: DataX__ServiceEnvironment__CosmosDBConfigDatabaseName + value: "" + - name: DataX__ServiceEnvironment__ServiceKeyVaultName + value: "" + - name: JwtBearerOptions__Audience + value: "" + - name: JwtBearerOptions__Authority + value: "" + - name: servicedllname + value: "Flow.LiveDataService.dll" + - name: DataX__ServiceEnvironment__EnableOneBox + value: "false" +--- +apiVersion: v1 +kind: Service +metadata: + name: flow-live-data-service +spec: + type: LoadBalancer + ports: + - port: 5000 + selector: + app: flow-live-data-service diff --git a/DeploymentCloud/Deployment.Kubernetes/Flow.ManagementService/flow-management-service.yaml b/DeploymentCloud/Deployment.Kubernetes/flow-management-service.yaml similarity index 84% rename from DeploymentCloud/Deployment.Kubernetes/Flow.ManagementService/flow-management-service.yaml rename to DeploymentCloud/Deployment.Kubernetes/flow-management-service.yaml index 073689df6..d1ee38181 100644 --- a/DeploymentCloud/Deployment.Kubernetes/Flow.ManagementService/flow-management-service.yaml +++ b/DeploymentCloud/Deployment.Kubernetes/flow-management-service.yaml @@ -21,7 +21,7 @@ spec: "beta.kubernetes.io/os": linux containers: - name: flow-management-service - image: dataxregistry.azurecr.io/datax/cloud:v1 + image: dataxregistry.azurecr.io/datax/cloud/flow.managementservice:v1 imagePullPolicy: "Always" ports: - containerPort: 5000 @@ -33,7 +33,7 @@ spec: env: - name: ASPNETCORE_URLS value: "http://+:5000" - - name: DataX__ServiceEnvironment__AppInsightsIntrumentationKeySecretName + - name: DataX__ServiceEnvironment__AppInsightsIntrumentationKey value: "" - name: DataX__ServiceEnvironment__CosmosDBConfigCollectionName value: "" @@ -46,7 +46,11 @@ spec: - name: JwtBearerOptions__Audience value: "" - name: JwtBearerOptions__Authority - value: "" + value: "" + - name: servicedllname + value: "Flow.ManagementService.dll" + - name: DataX__ServiceEnvironment__EnableOneBox + value: "false" --- apiVersion: v1 kind: Service diff --git a/DeploymentCloud/Deployment.Kubernetes/flow-schema-inference-service.yaml b/DeploymentCloud/Deployment.Kubernetes/flow-schema-inference-service.yaml new file mode 100644 index 000000000..080012d19 --- /dev/null +++ b/DeploymentCloud/Deployment.Kubernetes/flow-schema-inference-service.yaml @@ -0,0 +1,64 @@ +--- +apiVersion: apps/v1beta1 +kind: Deployment +metadata: + name: flow-schema-inference-service +spec: + replicas: 1 + strategy: + rollingUpdate: + maxSurge: 1 + maxUnavailable: 1 + minReadySeconds: 5 + template: + metadata: + labels: + app: flow-schema-inference-service + aadpodidbinding: managed-service-identity + spec: + terminationGracePeriodSeconds: 30 + nodeSelector: + "beta.kubernetes.io/os": linux + containers: + - name: flow-schema-inference-service + image: dataxregistry.azurecr.io/datax/cloud/flow.schemainferenceservice:v1 + imagePullPolicy: "Always" + ports: + - containerPort: 5000 + resources: + requests: + cpu: 250m + limits: + cpu: 500m + env: + - name: ASPNETCORE_URLS + value: "http://+:5000" + - name: DataX__ServiceEnvironment__AppInsightsIntrumentationKey + value: "" + - name: DataX__ServiceEnvironment__CosmosDBConfigCollectionName + value: "" + - name: DataX__ServiceEnvironment__CosmosDBConfigConnectionString + value: "" + - name: DataX__ServiceEnvironment__CosmosDBConfigDatabaseName + value: "" + - name: DataX__ServiceEnvironment__ServiceKeyVaultName + value: "" + - name: JwtBearerOptions__Audience + value: "" + - name: JwtBearerOptions__Authority + value: "" + - name: servicedllname + value: "Flow.SchemaInferenceService.dll" + - name: DataX__ServiceEnvironment__EnableOneBox + value: "false" +--- +apiVersion: v1 +kind: Service +metadata: + name: flow-schema-inference-service +spec: + type: LoadBalancer + ports: + - port: 5000 + selector: + app: flow-schema-inference-service diff --git a/DeploymentLocal/finalrun.sh b/DeploymentLocal/finalrun.sh index f6fef36fb..e1d1cbc57 100644 --- a/DeploymentLocal/finalrun.sh +++ b/DeploymentLocal/finalrun.sh @@ -1,3 +1,3 @@ #!/bin/bash -dotnet DataX.FlowManagement.dll & +dotnet Flow.ManagementService.dll & npm start diff --git a/DeploymentLocal/sample/HomeAutomationLocal.json b/DeploymentLocal/sample/HomeAutomationLocal.json index 0b4d142d1..b596d2227 100644 --- a/DeploymentLocal/sample/HomeAutomationLocal.json +++ b/DeploymentLocal/sample/HomeAutomationLocal.json @@ -316,7 +316,7 @@ "properties": { "name": null, - "path": "/app/aspnetcore/datax/bin/datax-udf-samples_2.3_2.11-1.1.0.jar", + "path": "/app/aspnetcore/datax/bin/datax-udf-samples_2.4_2.11-1.2.0.jar", "class": "datax.sample.udf.UdfHelloWorld", "libs": [] }, diff --git a/Services/.gitignore b/Services/.gitignore index 5f74ee22d..27f1c8bf6 100644 --- a/Services/.gitignore +++ b/Services/.gitignore @@ -128,4 +128,7 @@ BundleArtifacts/ /DeploymentCloud/Deployment.DataX/cachedVariables /DeploymentCloud/Deployment.DataX/Temp /DeploymentCloud/Deployment.DataX/Outputs -/DeploymentCloud/Deployment.DataX/Packages \ No newline at end of file +/DeploymentCloud/Deployment.DataX/Packages + +# Development settings +appsettings.Development.json \ No newline at end of file diff --git a/Services/AspnetCore/DataX.FlowManagement/DataX.FlowManagement/DataX.FlowManagement.csproj b/Services/AspnetCore/DataX.FlowManagement/DataX.FlowManagement/DataX.FlowManagement.csproj index 3208481ff..7bdf2950d 100644 --- a/Services/AspnetCore/DataX.FlowManagement/DataX.FlowManagement/DataX.FlowManagement.csproj +++ b/Services/AspnetCore/DataX.FlowManagement/DataX.FlowManagement/DataX.FlowManagement.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 AnyCPU true FinalPublicKey.snk @@ -14,15 +14,15 @@ - - - all - runtime; build; native; contentfiles; analyzers + + + all + runtime; build; native; contentfiles; analyzers - Microsoft + Microsoft400 StrongName diff --git a/Services/AspnetCore/DataX.FlowManagement/DataX.FlowManagement/Startup.cs b/Services/AspnetCore/DataX.FlowManagement/DataX.FlowManagement/Startup.cs index e0bf3c25c..301a1c0f0 100644 --- a/Services/AspnetCore/DataX.FlowManagement/DataX.FlowManagement/Startup.cs +++ b/Services/AspnetCore/DataX.FlowManagement/DataX.FlowManagement/Startup.cs @@ -27,8 +27,9 @@ public class Startup { private const string _MetricsHttpEndpointRelativeUri = "/api/data/upload"; + private readonly ILoggerFactory _loggerFactory; - public Startup(IHostingEnvironment env) + public Startup(IHostingEnvironment env, ILoggerFactory loggerFactory) { var builder = new ConfigurationBuilder() .SetBasePath(env.ContentRootPath) @@ -37,6 +38,8 @@ public Startup(IHostingEnvironment env) .AddEnvironmentVariables(); Configuration = builder.Build(); + + _loggerFactory = loggerFactory; } public IConfiguration Configuration { get; } @@ -51,7 +54,7 @@ public void ConfigureServices(IServiceCollection services) // Export the Config dependencies Type[] exportTypes = new Type[] { typeof(FlowOperation), typeof(RuntimeConfigGeneration), typeof(JobOperation) }; - services.AddMefExportsFromAssemblies(ServiceLifetime.Scoped, GetOneBoxModeDependencyAssemblies(), exportTypes, new object[] { }); + services.AddMefExportsFromAssemblies(ServiceLifetime.Scoped, GetOneBoxModeDependencyAssemblies(), exportTypes, null, _loggerFactory, true); var result = InitTemplatesForLocal(services); Ensure.IsSuccessResult(result); } diff --git a/Services/DataX.Config/CommonData.Templates/defaultSparkJob.json b/Services/DataX.Config/CommonData.Templates/defaultSparkJob.json index 3e54877de..997f327b6 100644 --- a/Services/DataX.Config/CommonData.Templates/defaultSparkJob.json +++ b/Services/DataX.Config/CommonData.Templates/defaultSparkJob.json @@ -3,16 +3,16 @@ "cluster": "${sparkClusterName}", "options": { "name": "${sparkJobName}", - "file": "wasbs:///datax/bin/datax-host_2.3_2.11-1.1.0.jar", + "file": "wasbs:///datax/bin/datax-host_2.4_2.11-1.2.0.jar", "className": "datax.app.DataXDirect", "args": [ "conf=${sparkJobConfigFilePath}", "driverLogLevel=${sparkJobDriverLogLevel}" ], "jars": [ - "wasbs:///datax/bin/datax-core_2.3_2.11-1.1.0.jar", - "wasbs:///datax/bin/datax-utility_2.3_2.11-1.1.0.jar", - "wasbs:///datax/bin/datax-keyvault_2.3_2.11-1.1.0-with-dependencies.jar", + "wasbs:///datax/bin/datax-core_2.4_2.11-1.2.0.jar", + "wasbs:///datax/bin/datax-utility_2.4_2.11-1.2.0.jar", + "wasbs:///datax/bin/datax-keyvault_2.4_2.11-1.2.0-with-dependencies.jar", "wasbs:///datax/bin/applicationinsights-core-2.2.1.jar", "wasbs:///datax/bin/azure-documentdb-1.16.1.jar", "wasbs:///datax/bin/azure-eventhubs-1.2.1.jar", diff --git a/Services/DataX.Config/DataX.Config.ConfigurationProviders/DataX.Config.ConfigurationProviders.csproj b/Services/DataX.Config/DataX.Config.ConfigurationProviders/DataX.Config.ConfigurationProviders.csproj index 98dcf1023..e170f4da2 100644 --- a/Services/DataX.Config/DataX.Config.ConfigurationProviders/DataX.Config.ConfigurationProviders.csproj +++ b/Services/DataX.Config/DataX.Config.ConfigurationProviders/DataX.Config.ConfigurationProviders.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -28,7 +28,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient.Test/DataX.Config.DatabricksClient.Test.csproj b/Services/DataX.Config/DataX.Config.DatabricksClient.Test/DataX.Config.DatabricksClient.Test.csproj new file mode 100644 index 000000000..e6fcf09a8 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient.Test/DataX.Config.DatabricksClient.Test.csproj @@ -0,0 +1,19 @@ + + + + netcoreapp2.2 + + false + + + + + + + + + + + + + diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient.Test/DatabricksClientTest.cs b/Services/DataX.Config/DataX.Config.DatabricksClient.Test/DatabricksClientTest.cs new file mode 100644 index 000000000..f0e87e23c --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient.Test/DatabricksClientTest.cs @@ -0,0 +1,81 @@ +using Microsoft.VisualStudio.TestTools.UnitTesting; +using Newtonsoft.Json.Linq; + +namespace DataX.Config.DatabricksClient.Test +{ + [TestClass] + public class DatabricksClientTest + { + [TestMethod] + public void TestParseJobInfoFromDatabricksHttpResult() + { + DatabricksHttpResult httpResult = new DatabricksHttpResult + { + Content = "{\"job_id\":40,\"run_id\":49,\"number_in_job\":1,\"original_attempt_run_id\":49,\"state\":{\"life_cycle_state\":\"RUNNING\",\"state_message\":\"In run\"}}", + IsSuccess = true, + StatusCode = System.Net.HttpStatusCode.OK + }; + var expectedValue = new SparkJobSyncResult + { + ClientCache = JToken.Parse("{\"job_id\": 40,\"run_id\": 49,\"state\": {\"life_cycle_state\": \"RUNNING\",\"state_message\": \"In run\"}}"), + JobId = "40", + JobState = JobState.Running, + Links = null, + Note = "In run" + }; + var actualValue = DatabricksClient.ParseJobInfoFromDatabricksHttpResult(httpResult); + Assert.AreEqual(expectedValue, actualValue, "ParseJobInfoFromDatabricksHttpResult() failed"); + } + + [TestMethod] + public void TestParseDatabricksJobResult() + { + var jobResult = new DatabricksJobResult + { + JobId = 40, + RunId = 49, + State = new System.Collections.Generic.Dictionary() + { + {"life_cycle_state","RUNNING"}, + {"state_message","In run"} + } + }; + var expectedValue = new SparkJobSyncResult + { + ClientCache = JToken.Parse("{\"job_id\": 40,\"run_id\": 49,\"state\": {\"life_cycle_state\": \"RUNNING\",\"state_message\": \"In run\"}}"), + JobId = "40", + JobState = JobState.Running, + Links = null, + Note = "In run" + }; + var actualValue = DatabricksClient.ParseDatabricksJobResult(jobResult); + Assert.AreEqual(expectedValue, actualValue, "ParseDatabricksJobResult() failed"); + } + + [TestMethod] + public void TestParseDatabricksJobState() + { + var expectedValue = JobState.Running; + var actualValue = DatabricksClient.ParseDatabricksJobState("RUNNING"); + Assert.AreEqual(expectedValue, actualValue, "State mismatch when job state is RUNNING"); + + expectedValue = JobState.Starting; + actualValue = DatabricksClient.ParseDatabricksJobState("PENDING"); + Assert.AreEqual(expectedValue, actualValue, "State mismatch when job state is PENDING"); + + expectedValue = JobState.Error; + actualValue = DatabricksClient.ParseDatabricksJobState("INTERNAL_ERROR"); + Assert.AreEqual(expectedValue, actualValue, "State mismatch when job state is INTERNAL_ERROR"); + + expectedValue = JobState.Idle; + actualValue = DatabricksClient.ParseDatabricksJobState("SKIPPED"); + Assert.AreEqual(expectedValue, actualValue, "State mismatch when job state is SKIPPED"); + + actualValue = DatabricksClient.ParseDatabricksJobState("TERMINATING"); + Assert.AreEqual(expectedValue, actualValue, "State mismatch when job state is TERMINATING"); + + actualValue = DatabricksClient.ParseDatabricksJobState("TERMINATED"); + Assert.AreEqual(expectedValue, actualValue, "State mismatch when job state is TERMINATED"); + } + } +} diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient/ConnectionStringParser.cs b/Services/DataX.Config/DataX.Config.DatabricksClient/ConnectionStringParser.cs new file mode 100644 index 000000000..958bda73e --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient/ConnectionStringParser.cs @@ -0,0 +1,37 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Contract.Exception; +using System; +using System.Collections.Generic; +using System.Text; +using System.Text.RegularExpressions; + +namespace DataX.Config.DatabricksClient +{ + public static class ConnectionStringParser + { + private static readonly Regex _DatabricksConnectionStringFormat = new Regex("^endpoint=([^;]*);dbtoken=(.*)$"); + + public static DatabricksClientConnectionInfo Parse(string connectionString) + { + if (connectionString == null) + { + throw new GeneralException($"connection string for livy client cannot be null"); + } + + var match = _DatabricksConnectionStringFormat.Match(connectionString); + if (match == null || !match.Success) + { + throw new GeneralException($"cannot parse connection string to access livy service"); + } + + return new DatabricksClientConnectionInfo() + { + Endpoint = match.Groups[1].Value, + DbToken = match.Groups[2].Value + }; + } + } +} diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient/DataX.Config.DatabricksClient.csproj b/Services/DataX.Config/DataX.Config.DatabricksClient/DataX.Config.DatabricksClient.csproj new file mode 100644 index 000000000..b0643f1d9 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient/DataX.Config.DatabricksClient.csproj @@ -0,0 +1,25 @@ + + + + netcoreapp2.2 + true + FinalPublicKey.snk + true + + + + + + + + + Microsoft400 + StrongName + + + + + + + + diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksClient.cs b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksClient.cs new file mode 100644 index 000000000..2372b51ee --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksClient.cs @@ -0,0 +1,173 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Contract; +using DataX.Contract.Exception; +using Newtonsoft.Json; +using Newtonsoft.Json.Linq; +using System; +using System.Collections.Generic; +using System.Net; +using System.Net.Http; +using System.Net.Http.Headers; +using System.Text; +using System.Threading.Tasks; + +namespace DataX.Config.DatabricksClient +{ + /// + /// Class to interact directly with the HTTP endpoint of the Databricks + /// + public class DatabricksClient : ISparkJobClient + { + private readonly DatabricksClientConnectionInfo _connectionInfo; + private readonly IDatabricksHttpClient _httpClient; + + public DatabricksClient(string connectionString, IDatabricksHttpClientFactory httpClientFactory) + { + _connectionInfo = ConnectionStringParser.Parse(connectionString); + _httpClient = httpClientFactory.CreateClientWithBearerToken(_connectionInfo.DbToken); + } + + /// + /// Call Databricks service to manage jobs + /// + /// GET, POST or DELETE + /// databricks api + /// body of httpRequest + /// Result with IsSuccess and message; exception message if exception occurs + private async Task CallDatabricksService(HttpMethod method, string api, string body = "") + { + //Make the call to the HTTP endpoint + var uri = new Uri(_connectionInfo.Endpoint + api); + return await _httpClient.ExecuteHttpRequest(method, uri, body); + } + + public static SparkJobSyncResult ParseJobInfoFromDatabricksHttpResult(DatabricksHttpResult httpResult) + { + if (httpResult.IsSuccess) + { + try + { + var job = JsonConvert.DeserializeObject(httpResult.Content); + return ParseDatabricksJobResult(job); + } + catch (Exception ex) + { + throw new GeneralException($"Couldn't parse response from Databricks service:'{httpResult.Content}', message:'{ex.Message}'"); + } + } + else if (httpResult.StatusCode == HttpStatusCode.NotFound) + { + // if session is not found, we should reset the state to allow user start a new job + return GetDefaultSparkJobSyncResult(httpResult.Content); + } + else + { + throw new GeneralException($"unexpected response from Databricks service:'{httpResult.StatusCode}', message:'{httpResult.Content}'"); + } + } + + private static SparkJobSyncResult GetDefaultSparkJobSyncResult(string content = "") + { + return new SparkJobSyncResult() + { + JobId = null, + JobState = JobState.Idle, + Note = content, + Links = null, + ClientCache = null + }; + } + + public static JobState ParseDatabricksJobState(string state) + { + switch (state) + { + case "PENDING": + return JobState.Starting; + case "RUNNING": + return JobState.Running; + case "INTERNAL_ERROR": + return JobState.Error; + case "SKIPPED": + case "TERMINATING": + case "TERMINATED": + return JobState.Idle; + default: + throw new GeneralException($"Unexpected databricks job state:'{state}'"); + } + } + + public static SparkJobSyncResult ParseDatabricksJobResult(DatabricksJobResult jobResult) + { + SparkJobSyncResult sr = new SparkJobSyncResult(); + + try + { + sr.JobId = jobResult.JobId.ToString(); + sr.JobState = ParseDatabricksJobState(jobResult.State.GetOrDefault("life_cycle_state", null)); + sr.ClientCache = JObject.FromObject(jobResult); + sr.Note = jobResult.State.GetOrDefault("state_message", null); + } + catch (Exception ex) + { + throw new GeneralException($"Couldn't parse response from Databricks service:'{jobResult}', message:'{ex.Message}'"); + } + return sr; + } + + public async Task SubmitJob(JToken jobData) + { + Ensure.NotNull(jobData, "jobData"); + JObject obj = JObject.Parse(jobData.ToString()); + JObject newCluster = (JObject)obj["new_cluster"]; + if (!(bool)obj.SelectToken("new_cluster.enableAutoscale")) + { + newCluster.Property("autoscale").Remove(); + } + else + { + newCluster.Property("num_workers").Remove(); + } + var jobResult = await CallDatabricksService(HttpMethod.Post, "jobs/create", obj.ToString()); + var jobId = JsonConvert.DeserializeObject(jobResult.Content).JobId; + var runResult = await CallDatabricksService(HttpMethod.Post, "jobs/run-now", $@"{{""job_id"":{jobId}}}"); + var runId = JsonConvert.DeserializeObject(runResult.Content).RunId; + var result = await CallDatabricksService(HttpMethod.Get, "jobs/runs/get?run_id=" + runId); + return ParseJobInfoFromDatabricksHttpResult(result); + } + + public async Task GetJobInfo(JToken jobClientData) + { + var runId = JsonConvert.DeserializeObject(jobClientData.ToString()).RunId; + var result = await CallDatabricksService(HttpMethod.Get, "jobs/runs/get?run_id=" + runId); + return ParseJobInfoFromDatabricksHttpResult(result); + } + + public async Task StopJob(JToken jobClientData) + { + var clientData = JsonConvert.DeserializeObject(jobClientData.ToString()); + await CallDatabricksService(HttpMethod.Post, "jobs/runs/cancel", $@"{{""run_id"":{clientData.RunId}}}"); + await CallDatabricksService(HttpMethod.Post, "jobs/delete", $@"{{""job_id"":{clientData.JobId}}}"); + var result = new SparkJobSyncResult(); + //Fetch status of job after it has been stopped + var numRetry = 0; + do + { + var jobStatus = await CallDatabricksService(HttpMethod.Get, $"jobs/runs/get?run_id={clientData.RunId}"); + result = ParseJobInfoFromDatabricksHttpResult(jobStatus); + //When job is in progress of termination, fetch the latest job state max 5 times untill the job has stopped. + numRetry++; + } while (result.JobState == JobState.Running && numRetry <= 5); + return result; + } + + public async Task GetJobs() + { + await Task.CompletedTask; + throw new NotImplementedException(); + } + } +} diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksClientConnectionInfo.cs b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksClientConnectionInfo.cs new file mode 100644 index 000000000..1e0be9b6f --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksClientConnectionInfo.cs @@ -0,0 +1,12 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +namespace DataX.Config.DatabricksClient +{ + public class DatabricksClientConnectionInfo + { + public string Endpoint { get; set; } + public string DbToken { get; set; } + } +} diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksClientFactory.cs b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksClientFactory.cs new file mode 100644 index 000000000..9d39e182c --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksClientFactory.cs @@ -0,0 +1,29 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using System; +using System.Collections.Generic; +using System.Composition; +using System.Text; +using System.Threading.Tasks; + +namespace DataX.Config.DatabricksClient +{ + [Export(typeof(ISparkJobClientFactory))] + public class DatabricksClientFactory : ISparkJobClientFactory + { + [ImportingConstructor] + public DatabricksClientFactory(IDatabricksHttpClientFactory httpClientFactory) + { + this.HttpClientFactory = httpClientFactory; + } + + private IDatabricksHttpClientFactory HttpClientFactory { get; } + + public Task GetClient(string connectionString) + { + return Task.FromResult((ISparkJobClient)new DatabricksClient(connectionString, HttpClientFactory)); + } + } +} diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksHttpClient.cs b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksHttpClient.cs new file mode 100644 index 000000000..982899ffe --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksHttpClient.cs @@ -0,0 +1,57 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Contract.Exception; +using System; +using System.Collections.Generic; +using System.Net.Http; +using System.Text; +using System.Threading.Tasks; + +namespace DataX.Config.DatabricksClient +{ + public class DatabricksHttpClient : IDatabricksHttpClient + { + private HttpClient _client; + + public DatabricksHttpClient(HttpClient client) + { + _client = client; + } + + public async Task ExecuteHttpRequest(HttpMethod method, Uri uri, string body = "") + { + try + { + var response = await ExecuteHttpRequestInternal(method, uri, body); + var message = await response.Content.ReadAsStringAsync(); + return new DatabricksHttpResult() + { + IsSuccess = response.IsSuccessStatusCode, + StatusCode = response.StatusCode, + Content = message + }; + } + catch (Exception ex) + { + throw new GeneralException($"failed in calling '{uri}' due to error:{ex.Message}"); + } + } + + public Task ExecuteHttpRequestInternal(HttpMethod method, Uri uri, string body = "") + { + switch (method) + { + case HttpMethod.Get: + return _client.GetAsync(uri); + case HttpMethod.Post: + return _client.PostAsync(uri, new StringContent(body, Encoding.UTF8, "application/json")); + case HttpMethod.Delete: + return _client.DeleteAsync(uri); + default: + throw new GeneralException($"Unsupported method :'{method}'"); + } + } + } +} diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksHttpClientFactory.cs b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksHttpClientFactory.cs new file mode 100644 index 000000000..e06e2c4f7 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksHttpClientFactory.cs @@ -0,0 +1,27 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using System; +using System.Collections.Generic; +using System.Composition; +using System.Net.Http; +using System.Net.Http.Headers; +using System.Text; + +namespace DataX.Config.DatabricksClient +{ + [Shared] + [Export(typeof(IDatabricksHttpClientFactory))] + public class DatabricksHttpClientFactory : IDatabricksHttpClientFactory + { + public IDatabricksHttpClient CreateClientWithBearerToken(string dbToken) + { + var client = new HttpClient(); + client.DefaultRequestHeaders.Accept.Clear(); + client.DefaultRequestHeaders.Accept.Add(new MediaTypeWithQualityHeaderValue("application/json")); + client.DefaultRequestHeaders.Authorization = new AuthenticationHeaderValue("Bearer", dbToken); + return new DatabricksHttpClient(client); + } + } +} diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksHttpResult.cs b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksHttpResult.cs new file mode 100644 index 000000000..1a77973a6 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksHttpResult.cs @@ -0,0 +1,18 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using System; +using System.Collections.Generic; +using System.Net; +using System.Text; + +namespace DataX.Config.DatabricksClient +{ + public class DatabricksHttpResult + { + public HttpStatusCode StatusCode; + public string Content; + public bool IsSuccess; + } +} diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksJobResult.cs b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksJobResult.cs new file mode 100644 index 000000000..3a11bf09a --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient/DatabricksJobResult.cs @@ -0,0 +1,23 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Newtonsoft.Json; +using System; +using System.Collections.Generic; +using System.Text; + +namespace DataX.Config.DatabricksClient +{ + public class DatabricksJobResult + { + [JsonProperty("job_id")] + public int JobId { get; set; } + + [JsonProperty("run_id")] + public int RunId { get; set; } + + [JsonProperty("state")] + public Dictionary State { get; set; } + } +} diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient/FinalPublicKey.snk b/Services/DataX.Config/DataX.Config.DatabricksClient/FinalPublicKey.snk new file mode 100644 index 000000000..110b59c7b Binary files /dev/null and b/Services/DataX.Config/DataX.Config.DatabricksClient/FinalPublicKey.snk differ diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient/HttpMethod.cs b/Services/DataX.Config/DataX.Config.DatabricksClient/HttpMethod.cs new file mode 100644 index 000000000..94aaf4420 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient/HttpMethod.cs @@ -0,0 +1,17 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using System; +using System.Collections.Generic; +using System.Text; + +namespace DataX.Config.DatabricksClient +{ + public enum HttpMethod + { + Get, + Post, + Delete + } +} diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient/IDatabricksHttpClient.cs b/Services/DataX.Config/DataX.Config.DatabricksClient/IDatabricksHttpClient.cs new file mode 100644 index 000000000..1bfef6ff1 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient/IDatabricksHttpClient.cs @@ -0,0 +1,16 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using System; +using System.Collections.Generic; +using System.Text; +using System.Threading.Tasks; + +namespace DataX.Config.DatabricksClient +{ + public interface IDatabricksHttpClient + { + Task ExecuteHttpRequest(HttpMethod method, Uri uri, string body = ""); + } +} diff --git a/Services/DataX.Config/DataX.Config.DatabricksClient/IDatabricksHttpClientFactory.cs b/Services/DataX.Config/DataX.Config.DatabricksClient/IDatabricksHttpClientFactory.cs new file mode 100644 index 000000000..9efb64ee4 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.DatabricksClient/IDatabricksHttpClientFactory.cs @@ -0,0 +1,15 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using System; +using System.Collections.Generic; +using System.Text; + +namespace DataX.Config.DatabricksClient +{ + public interface IDatabricksHttpClientFactory + { + IDatabricksHttpClient CreateClientWithBearerToken(string dbToken); + } +} diff --git a/Services/DataX.Config/DataX.Config.Input.EventHub/DataX.Config.Input.EventHub.csproj b/Services/DataX.Config/DataX.Config.Input.EventHub/DataX.Config.Input.EventHub.csproj index 149ef46d2..dd32e99a3 100644 --- a/Services/DataX.Config/DataX.Config.Input.EventHub/DataX.Config.Input.EventHub.csproj +++ b/Services/DataX.Config/DataX.Config.Input.EventHub/DataX.Config.Input.EventHub.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -28,7 +28,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Config/DataX.Config.Input.EventHub/Processor/CreateEventHubConsumerGroup.cs b/Services/DataX.Config/DataX.Config.Input.EventHub/Processor/CreateEventHubConsumerGroup.cs index 001b16c6b..44fc6463e 100644 --- a/Services/DataX.Config/DataX.Config.Input.EventHub/Processor/CreateEventHubConsumerGroup.cs +++ b/Services/DataX.Config/DataX.Config.Input.EventHub/Processor/CreateEventHubConsumerGroup.cs @@ -47,15 +47,16 @@ public CreateEventHubConsumerGroup(ConfigGenConfiguration configuration, IKeyVau public override async Task HandleSensitiveData(FlowGuiConfig guiConfig) { var runtimeKeyVaultName = Configuration[Constants.ConfigSettingName_RuntimeKeyVaultName]; + var sparkType = Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string value) ? value : null; Ensure.NotNull(runtimeKeyVaultName, "runtimeKeyVaultName"); // Replace Input Event Hub Connection String var eventHubConnectionString = guiConfig?.Input?.Properties?.InputEventhubConnection; - if (eventHubConnectionString != null && !KeyVaultUri.IsSecretUri(eventHubConnectionString)) + if (!string.IsNullOrEmpty(eventHubConnectionString) && !KeyVaultUri.IsSecretUri(eventHubConnectionString)) { - //TODO: create new secret + // create new secret var secretName = $"{guiConfig.Name}-input-eventhubconnectionstring"; - var secretId = await KeyVaultClient.SaveSecretAsync(runtimeKeyVaultName, secretName, eventHubConnectionString); + var secretId = await KeyVaultClient.SaveSecretAsync(runtimeKeyVaultName, secretName, eventHubConnectionString, sparkType); guiConfig.Input.Properties.InputEventhubConnection = secretId; } @@ -64,7 +65,7 @@ public override async Task HandleSensitiveData(FlowGuiConfig guiC if (!string.IsNullOrEmpty(inputSubscriptionId) && !KeyVaultUri.IsSecretUri(inputSubscriptionId)) { var secretName = $"{guiConfig.Name}-input-inputsubscriptionid"; - var secretId = await KeyVaultClient.SaveSecretAsync(runtimeKeyVaultName, secretName, inputSubscriptionId); + var secretId = await KeyVaultClient.SaveSecretAsync(runtimeKeyVaultName, secretName, inputSubscriptionId, sparkType); guiConfig.Input.Properties.InputSubscriptionId = secretId; } @@ -73,10 +74,19 @@ public override async Task HandleSensitiveData(FlowGuiConfig guiC if (!string.IsNullOrEmpty(inputResourceGroup) && !KeyVaultUri.IsSecretUri(inputResourceGroup)) { var secretName = $"{guiConfig.Name}-input-inputResourceGroup"; - var secretId = await KeyVaultClient.SaveSecretAsync(runtimeKeyVaultName, secretName, inputResourceGroup); + var secretId = await KeyVaultClient.SaveSecretAsync(runtimeKeyVaultName, secretName, inputResourceGroup, sparkType); guiConfig.Input.Properties.InputResourceGroup = secretId; } + // Replace Info Databricks Token + var infoDatabricksToken = guiConfig?.DatabricksToken; + if(!string.IsNullOrEmpty(infoDatabricksToken) && !KeyVaultUri.IsSecretUri(infoDatabricksToken)) + { + var secretName = $"{guiConfig.Name}-info-databricksToken"; + var secretId = await KeyVaultClient.SaveSecretAsync(runtimeKeyVaultName, secretName, infoDatabricksToken, sparkType); + guiConfig.DatabricksToken = secretId; + } + return guiConfig; } @@ -92,7 +102,7 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) return "eventhub/iothub input not defined, skipped"; } - if (inputType != Constants.InputType_EventHub && inputType != Constants.InputType_IoTHub && inputType != Constants.InputType_KafkaEventHub && inputType != Constants.InputType_Kafka) + if (inputType != Constants.InputType_EventHub && inputType != Constants.InputType_IoTHub && inputType != Constants.InputType_KafkaEventHub && inputType != Constants.InputType_Kafka && inputType != Constants.InputType_Blob) { return $"unsupported inputtype '{inputType}', skipped."; } @@ -170,8 +180,8 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) flowToDeploy.SetStringToken(TokenName_InputEventHubConsumerGroup, consumerGroupName); flowToDeploy.SetStringToken(TokenName_InputEventHubs, hubInfo.Name); - - var checkpointDir = Configuration.GetOrDefault(ConfigSettingName_InputEventHubCheckpointDir, "hdfs://mycluster/datax/direct/${name}/"); + var sparkType = Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string value) ? value : null; + var checkpointDir = (sparkType == Constants.SparkTypeDataBricks) ? Configuration.GetOrDefault(ConfigSettingName_InputEventHubCheckpointDir, "dbfs:/mycluster/datax/direct/${name}/") : Configuration.GetOrDefault(ConfigSettingName_InputEventHubCheckpointDir, "hdfs://mycluster/datax/direct/${name}/"); flowToDeploy.SetStringToken(TokenName_InputEventHubCheckpointDir, checkpointDir); var intervalInSeconds = props?.WindowDuration; diff --git a/Services/DataX.Config/DataX.Config.KeyVault/DataX.Config.KeyVault.csproj b/Services/DataX.Config/DataX.Config.KeyVault/DataX.Config.KeyVault.csproj index e2ae1ae42..421d302f7 100644 --- a/Services/DataX.Config/DataX.Config.KeyVault/DataX.Config.KeyVault.csproj +++ b/Services/DataX.Config/DataX.Config.KeyVault/DataX.Config.KeyVault.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -26,7 +26,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Config/DataX.Config.KeyVault/KeyVaultClient.cs b/Services/DataX.Config/DataX.Config.KeyVault/KeyVaultClient.cs index b8b5a51dd..34df64150 100644 --- a/Services/DataX.Config/DataX.Config.KeyVault/KeyVaultClient.cs +++ b/Services/DataX.Config/DataX.Config.KeyVault/KeyVaultClient.cs @@ -2,6 +2,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using DataX.Config.ConfigDataModel; using DataX.Contract; using DataX.Utility.KeyVault; using System; @@ -35,17 +36,31 @@ public Task ResolveSecretUriAsync(string secretUri) return GetKeyVault().GetSecretFromKeyvaultAsync(secretUri); } - public async Task SaveSecretAsync(string keyvaultName, string secretUri, string secret) + public async Task SaveSecretAsync(string keyvaultName, string secretUri, string secret, string sparkType) { + var uriPrefix = (sparkType != null && sparkType == Constants.SparkTypeDataBricks) ? Constants.PrefixSecretScope : Constants.PrefixKeyVault; await GetKeyVault().SaveSecretStringAsync(keyvaultName, secretUri, secret); - return SecretUriParser.ComposeUri(keyvaultName, secretUri); + return SecretUriParser.ComposeUri(keyvaultName, secretUri, uriPrefix); } - public async Task SaveSecretAsync(string keyvaultName, string secretName, string secretValue, bool hashSuffix = false) + public async Task SaveSecretAsync(string keyvaultName, string secretName, string secretValue, string sparkType, bool hashSuffix = false) { var finalSecretName = hashSuffix ? (secretName + "-" + HashGenerator.GetHashCode(secretValue)) : secretName; + var uriPrefix = GetUriPrefix(sparkType); await GetKeyVault().SaveSecretStringAsync(keyvaultName, finalSecretName, secretValue); - return SecretUriParser.ComposeUri(keyvaultName, finalSecretName); + return SecretUriParser.ComposeUri(keyvaultName, finalSecretName, uriPrefix); + } + + public async Task SaveSecretAsync(string secretUri, string secretValue) + { + SecretUriParser.ParseSecretUri(secretUri, out string keyvaultName, out string secretName); + await GetKeyVault().SaveSecretStringAsync(keyvaultName, secretName, secretValue); + return secretUri; + } + + public string GetUriPrefix(string sparkType) + { + return (sparkType != null && sparkType == Constants.SparkTypeDataBricks) ? Constants.PrefixSecretScope : Constants.PrefixKeyVault; } } } diff --git a/Services/DataX.Config/DataX.Config.LivyClient.Test/DataX.Config.LivyClient.Test.csproj b/Services/DataX.Config/DataX.Config.LivyClient.Test/DataX.Config.LivyClient.Test.csproj index cd4d4c1a5..b67e3b6cc 100644 --- a/Services/DataX.Config/DataX.Config.LivyClient.Test/DataX.Config.LivyClient.Test.csproj +++ b/Services/DataX.Config/DataX.Config.LivyClient.Test/DataX.Config.LivyClient.Test.csproj @@ -1,7 +1,7 @@ - netcoreapp2.1 + netcoreapp2.2 false diff --git a/Services/DataX.Config/DataX.Config.LivyClient.Test/JobOperationTest.cs b/Services/DataX.Config/DataX.Config.LivyClient.Test/JobOperationTest.cs index 38a15c68d..8560033e1 100644 --- a/Services/DataX.Config/DataX.Config.LivyClient.Test/JobOperationTest.cs +++ b/Services/DataX.Config/DataX.Config.LivyClient.Test/JobOperationTest.cs @@ -36,6 +36,51 @@ public async Task TestStartJob() Assert.AreEqual(expected: JobState.Running, actual: refreshJobInfo.JobState); } + [TestMethod] + public async Task TestGetJobs() + { + var appInfoJson = "\"appInfo\":{\"driverLogUrl\":\"http://123.xx.internal.cloudapp.net:3/node/containerlogs/container_123/livy\",\"sparkUiUrl\":\"https://site.azurehdinsight.net/yarnui/hn/proxy/application_123_0101/\"}"; + var logs1 = ",\"log\":[\"\\t user: livy\",\"19/01/30 04:02:31 INFO ShutdownHookManager: Shutdown hook called\",\"\\nYARN Diagnostics: \"]}"; + var logs2 = ",\"log\":[\"\\t user: livy\",\"19/01/30 04:02:31 INFO ShutdownHookManager: Shutdown hook called\",\"19/01/30 04:02:31 INFO ShutdownHookManager: Deleting directory /tmp/spark-123\",\"19/01/30 04:02:31 INFO ShutdownHookManager: Deleting directory /tmp/spark-123\",\"19/01/30 04:02:31 INFO MetricsSystemImpl: Stopping azure-file-system metrics system...\",\"19/01/30 04:02:31 INFO MetricsSinkAdapter: azurefs2 thread interrupted.\",\"19/01/30 04:02:31 INFO MetricsSystemImpl: azure-file-system metrics system stopped.\",\"19/01/30 04:02:31 INFO MetricsSystemImpl: azure-file-system metrics system shutdown complete.\",\"\\nstderr: \",\"\\nYARN Diagnostics: \"]}"; + + var httpClientFactory = new HttpClientFactory(); + httpClientFactory + .AddResponse(HttpMethod.Get, "http://localhost/batches", "{\"from\":\"0\",\"total\":\"2\"," + "\"sessions\":[" + + "{\"id\":999,\"state\":\"starting\",\"appId\":\"application_123_0101\"," + appInfoJson + logs1 + "," + + "{\"id\":9991,\"state\":\"starting\",\"appId\":\"application_123_0102\"," + appInfoJson + logs2 + "" + + "]}"); + + var livyClientFactory = new LivyClientFactory(httpClientFactory); + var livyClient = await livyClientFactory.GetClient("endpoint=http://localhost;username=test;password=test"); + var jobInfo = await livyClient.GetJobs(); + + Assert.AreEqual(expected: 2, actual: jobInfo.Length); + + Assert.AreEqual(expected: JobState.Starting, actual: jobInfo[0].JobState); + Assert.AreEqual(expected: "999", actual: jobInfo[0].JobId); + + Assert.AreEqual(expected: JobState.Starting, actual: jobInfo[1].JobState); + Assert.AreEqual(expected: "9991", actual: jobInfo[1].JobId); + } + + [TestMethod] + public async Task TestStopJob() + { + var appInfoJson = "\"appInfo\":{\"driverLogUrl\":\"http://123.xx.internal.cloudapp.net:3/node/containerlogs/container_123/livy\",\"sparkUiUrl\":\"https://site.azurehdinsight.net/yarnui/hn/proxy/application_123_0101/\"}"; + var logs1 = ",\"log\":[\"\\t user: livy\",\"19/01/30 04:02:31 INFO ShutdownHookManager: Shutdown hook called\",\"\\nYARN Diagnostics: \"]}"; + + var httpClientFactory = new HttpClientFactory(); + httpClientFactory + .AddResponse(HttpMethod.Delete, "http://localhost/batches/0", "{\"id\":2,\"state\":\"starting\",\"appId\":\"application_123_0101\"," + appInfoJson + logs1); + + var livyClientFactory = new LivyClientFactory(httpClientFactory); + var livyClient = await livyClientFactory.GetClient("endpoint=http://localhost;username=test;password=test"); + + var jobDelete = await livyClient.StopJob(JObject.Parse("{}")); + + Assert.AreEqual(expected: null, actual: jobDelete.JobId); + } + [TestMethod] public async Task TestJobNotFound() { diff --git a/Services/DataX.Config/DataX.Config.LivyClient/DataX.Config.LivyClient.csproj b/Services/DataX.Config/DataX.Config.LivyClient/DataX.Config.LivyClient.csproj index 2a0867201..0f9f570b1 100644 --- a/Services/DataX.Config/DataX.Config.LivyClient/DataX.Config.LivyClient.csproj +++ b/Services/DataX.Config/DataX.Config.LivyClient/DataX.Config.LivyClient.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -25,7 +25,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Config/DataX.Config.LivyClient/LivyClient.cs b/Services/DataX.Config/DataX.Config.LivyClient/LivyClient.cs index 23d90cc7d..228cd5711 100644 --- a/Services/DataX.Config/DataX.Config.LivyClient/LivyClient.cs +++ b/Services/DataX.Config/DataX.Config.LivyClient/LivyClient.cs @@ -185,6 +185,7 @@ public static JobState ParseLivyBatchState(string state) case "starting": return JobState.Starting; case "running":return JobState.Running; case "dead":return JobState.Idle; + case "success":return JobState.Success; default: throw new GeneralException($"Unexpected livy batch state:'{state}'"); } } diff --git a/Services/DataX.Config/DataX.Config.Local.Test/DataX.Config.Local.Test.csproj b/Services/DataX.Config/DataX.Config.Local.Test/DataX.Config.Local.Test.csproj index dd650e813..c18fc9368 100644 --- a/Services/DataX.Config/DataX.Config.Local.Test/DataX.Config.Local.Test.csproj +++ b/Services/DataX.Config/DataX.Config.Local.Test/DataX.Config.Local.Test.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 false @@ -72,7 +72,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Config/DataX.Config.Local.Test/LocalTests.cs b/Services/DataX.Config/DataX.Config.Local.Test/LocalTests.cs index 94446171e..9ae0cf0dc 100644 --- a/Services/DataX.Config/DataX.Config.Local.Test/LocalTests.cs +++ b/Services/DataX.Config/DataX.Config.Local.Test/LocalTests.cs @@ -36,7 +36,8 @@ public static void Initialize(TestContext tc) var conf = new ContainerConfiguration() .WithAssembly(typeof(ConfigGenConfiguration).Assembly) - .WithAssembly(typeof(DataX.Config.Local.LocalDesignTimeStorage).Assembly); + .WithAssembly(typeof(DataX.Config.Local.LocalDesignTimeStorage).Assembly) + .WithProvider(new LoggerAndInstanceExportDescriptorProvider(null, new LoggerFactory())); CompositionHost = conf.CreateContainer(); _TestContext = tc; diff --git a/Services/DataX.Config/DataX.Config.Local.Test/LoggerFactory.cs b/Services/DataX.Config/DataX.Config.Local.Test/LoggerFactory.cs new file mode 100644 index 000000000..39befc39b --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Local.Test/LoggerFactory.cs @@ -0,0 +1,28 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Config.Test.Utility.Mock; +using Microsoft.Extensions.Logging; + +namespace DataX.Config.Test +{ + internal class LoggerFactory : ILoggerFactory + { + public void AddProvider(ILoggerProvider provider) + { + throw new System.NotImplementedException(); + } + + public ILogger CreateLogger(string categoryName) + { + var result = new CacheLogger(); + return result; + } + + public void Dispose() + { + throw new System.NotImplementedException(); + } + } +} diff --git a/Services/DataX.Config/DataX.Config.Local.Test/Resource/flowSaved.json b/Services/DataX.Config/DataX.Config.Local.Test/Resource/flowSaved.json index 05353e10a..994a8910b 100644 --- a/Services/DataX.Config/DataX.Config.Local.Test/Resource/flowSaved.json +++ b/Services/DataX.Config/DataX.Config.Local.Test/Resource/flowSaved.json @@ -56,7 +56,7 @@ "input": { "type": "MetricApi", "metricKeys": [ - "DATAX-localconfiggentest:Input_DataXProcessedInput_Events_Count" + "DATAX-${name}:Input_DataXProcessedInput_Events_Count" ] }, "output": { @@ -66,151 +66,8 @@ "timechart": true, "average": true, "speed": true - } - } - }, - { - "name": "HeaterStatus, WindowLockStatus, WindowLockSliding", - "input": { - "type": "MetricApi", - "pollingInterval": 60000, - "metricKeys": [ - { - "name": "localconfiggentest:HeaterStatus", - "displayName": "HeaterStatus" - }, - { - "name": "localconfiggentest:WindowLockStatus", - "displayName": "WindowLockStatus" - }, - { - "name": "localconfiggentest:WindowLockSliding", - "displayName": "WindowLockSliding" - } - ] - }, - "output": { - "type": "DirectTimeChart", - "data": { - "timechart": true, - "current": false, - "table": false - }, - "chartTimeWindowInMs": 3600000 - } - }, - { - "name": "WindowOpenWhileHeaterOnAlert", - "input": { - "type": "MetricDetailsApi", - "pollingInterval": 60000, - "metricKeys": [ - { - "name": "localconfiggentest:WindowOpenWhileHeaterOnAlert", - "displayName": "WindowOpenWhileHeaterOnAlert" - } - ] - }, - "output": { - "type": "DirectTable", - "data": { - "timechart": false, - "current": false, - "table": true }, - "chartTimeWindowInMs": 3600000 - } - }, - { - "name": "DoorLockStatus", - "input": { - "type": "MetricApi", - "pollingInterval": 60000, - "metricKeys": [ - { - "name": "localconfiggentest:DoorLockStatus", - "displayName": "DoorLockStatus" - } - ] - }, - "output": { - "type": "DirectTimeChart", - "data": { - "timechart": true, - "current": false, - "table": false - }, - "chartTimeWindowInMs": 3600000 - } - }, - { - "name": "DoorLockOpenBetween9pmAnd7amUtcAlert", - "input": { - "type": "MetricDetailsApi", - "pollingInterval": 60000, - "metricKeys": [ - { - "name": "localconfiggentest:DoorLockOpenBetween9pmAnd7amUtcAlert", - "displayName": "DoorLockOpenBetween9pmAnd7amUtcAlert" - } - ] - }, - "output": { - "type": "DirectTable", - "data": { - "timechart": false, - "current": false, - "table": true - }, - "chartTimeWindowInMs": 3600000 - } - }, - { - "name": "GarageDoorStatus, GarageDoorSliding", - "input": { - "type": "MetricApi", - "pollingInterval": 60000, - "metricKeys": [ - { - "name": "localconfiggentest:GarageDoorStatus", - "displayName": "GarageDoorStatus" - }, - { - "name": "localconfiggentest:GarageDoorSliding", - "displayName": "GarageDoorSliding" - } - ] - }, - "output": { - "type": "DirectTimeChart", - "data": { - "timechart": true, - "current": false, - "table": false - }, - "chartTimeWindowInMs": 3600000 - } - }, - { - "name": "GarageOpenFor5MinsInAnHourAlert", - "input": { - "type": "MetricDetailsApi", - "pollingInterval": 60000, - "metricKeys": [ - { - "name": "localconfiggentest:GarageOpenFor5MinsInAnHourAlert", - "displayName": "GarageOpenFor5MinsInAnHourAlert" - } - ] - }, - "output": { - "type": "DirectTable", - "data": { - "timechart": false, - "current": false, - "table": true - }, - "chartTimeWindowInMs": 3600000 + "dynamicOffsetInMs": 300000 } } ], @@ -237,48 +94,6 @@ "formatter": "longint", "position": "FirstRow", "type": "SimpleBox" - }, - { - "name": "HeaterStatus, WindowLockStatus, WindowLockSliding", - "displayName": "HeaterStatus, WindowLockStatus, WindowLockSliding", - "data": "HeaterStatus, WindowLockStatus, WindowLockSliding_timechart", - "position": "TimeCharts", - "type": "MultiLineChart" - }, - { - "name": "WindowOpenWhileHeaterOnAlert", - "displayName": "WindowOpenWhileHeaterOnAlert", - "data": "WindowOpenWhileHeaterOnAlert_table", - "position": "TimeCharts", - "type": "DetailsList" - }, - { - "name": "DoorLockStatus", - "displayName": "DoorLockStatus", - "data": "DoorLockStatus_timechart", - "position": "TimeCharts", - "type": "MultiLineChart" - }, - { - "name": "DoorLockOpenBetween9pmAnd7amUtcAlert", - "displayName": "DoorLockOpenBetween9pmAnd7amUtcAlert", - "data": "DoorLockOpenBetween9pmAnd7amUtcAlert_table", - "position": "TimeCharts", - "type": "DetailsList" - }, - { - "name": "GarageDoorStatus, GarageDoorSliding", - "displayName": "GarageDoorStatus, GarageDoorSliding", - "data": "GarageDoorStatus, GarageDoorSliding_timechart", - "position": "TimeCharts", - "type": "MultiLineChart" - }, - { - "name": "GarageOpenFor5MinsInAnHourAlert", - "displayName": "GarageOpenFor5MinsInAnHourAlert", - "data": "GarageOpenFor5MinsInAnHourAlert_table", - "position": "TimeCharts", - "type": "DetailsList" } ], "initParameters": { @@ -293,6 +108,7 @@ "gui": { "name": "localconfiggentest", "displayName": "localconfiggentest", + "databricksToken": null, "owner": "data-accelerator@microsoft.com", "input": { "mode": "streaming", @@ -307,7 +123,9 @@ "maxRate": "35000", "inputSchemaFile": "{\"type\":\"struct\",\"fields\":[{\"name\":\"deviceDetails\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"deviceId\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"deviceType\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"eventTime\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"homeId\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"status\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}", "showNormalizationSnippet": false, - "normalizationSnippet": "stringToTimestamp(Raw.deviceDetails.eventTime) AS eventTimeStamp\r\nRaw.*" + "normalizationSnippet": "stringToTimestamp(Raw.deviceDetails.eventTime) AS eventTimeStamp\r\nRaw.*", + "inputSubscriptionId": "", + "inputResourceGroup": "" }, "referenceData": [ { @@ -320,7 +138,8 @@ "header": true } } - ] + ], + "batch": null }, "process": { "timestampColumn": "eventTime", @@ -332,7 +151,7 @@ "properties": { "name": "myudaf", "path": "wasbs://sdfsdf/sdfsdf", - "class": "datax.udaf.myudaf", + "class": "datax.udf.myudaf", "libs": [] }, "typeDisplay": "UDAF" @@ -365,7 +184,10 @@ "queries": [ "--DataXStates--\nCREATE TABLE configgentest_garagedoor_accumulated\n (deviceId long, deviceType string, homeId long, MaxEventTime Timestamp, MinReading long, MaxReading long);\n\n--DataXQuery--\nNPI_DeviceWindow = SELECT \n deviceDetails.deviceId,\n deviceDetails.deviceType,\n eventTimeStamp,\n deviceDetails.homeId,\n deviceDetails.status\n FROM DataXProcessedInput \n TIMEWINDOW('5 minutes');\n\n--DataXQuery--\nNPI_DeviceWindow2 = SELECT \n deviceDetails.deviceId,\n deviceDetails.deviceType,\n eventTimeStamp,\n deviceDetails.homeId,\n deviceDetails.status\n FROM DataXProcessedInput \n TIMEWINDOW('2 minutes');\n\n--DataXQuery--\nDeviceInfoTimeWindow = SELECT \n deviceId,\n deviceType,\n homeId,\n MAX(eventTimeStamp) AS MaxEventTime,\n MIN(status) AS MinReading,\n MAX(status) AS MaxReading\n FROM NPI_DeviceWindow\n GROUP BY deviceId, deviceType, homeId;\n\n--DataXQuery--\nHeaterStatus = SELECT \n eventTimeStamp AS EventTime,\n 'HeaterStatus' AS MetricName,\n deviceDetails.status AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'Heating';\n \n--DataXQuery--\nWindowLockStatus = SELECT \n eventTimeStamp AS EventTime,\n 'WindowLockStatus' AS MetricName,\n deviceDetails.status AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'WindowLock';\n\n--DataXQuery--\nWindowLockSliding = SELECT\n MaxEventTime AS EventTime,\n 'WindowLockSliding' AS MetricName,\n MaxReading AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DeviceInfoTimeWindow\n INNER JOIN WindowLockStatus ON WindowLockStatus.EventTime = DeviceInfoTimeWindow.MaxEventTime\n WHERE homeId = 150\n\t\t\t\t\t\tAND deviceType = 'WindowLock';\n\n--DataXQuery--\nWindowOpenWhileHeaterOnAlert = SELECT\n MaxEventTime AS EventTime,\n 'WindowOpenWhileHeaterOnAlert' AS MetricName,\n 0 AS Metric,\n 'iotdevicesample' AS Product, \n 'Window open for 5+ minutes while heater is on.' AS Pivot1\n FROM DeviceInfoTimeWindow\n INNER JOIN WindowLockStatus ON WindowLockStatus.EventTime = DeviceInfoTimeWindow.MaxEventTime\n WHERE homeId = 150\n AND MaxReading = 0\n\t\t\t\t\t\t\t\t\tAND deviceType = 'WindowLock';\n\n--DataXQuery--\nDoorLockStatus = SELECT\n eventTimeStamp AS EventTime,\n 'DoorLockStatus' AS MetricName,\n deviceDetails.status AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'DoorLock';\n\n--DataXQuery--\nDoorLockOpenBetween9pmAnd7amUtcAlert = SELECT\n eventTimeStamp AS EventTime,\n 'DoorLockOpenBetween9pmAnd7amUtcAlert' AS MetricName,\n 0 AS Metric,\n 'iotdevicesample' AS Product, \n 'Door lock open between 9PM and 7AM UTC' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'DoorLock'\n AND deviceDetails.status = 0\n AND (hour(eventTimeStamp) >= 21 OR hour(eventTimeStamp) < 7);\n\n--DataXQuery--\nGarageDoorStatus = SELECT\n eventTimeStamp AS EventTime,\n 'GarageDoorStatus' AS MetricName,\n deviceDetails.status AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'GarageDoorLock';\n\n--DataXQuery--\nGarageDoorAccumalator = SELECT \n deviceId,\n deviceType,\n homeId,\n MaxEventTime,\n MinReading,\n MaxReading\n FROM DeviceInfoTimeWindow\n WHERE homeId = 150\n AND deviceType = 'GarageDoorLock'\n UNION ALL\n SELECT \n deviceId,\n deviceType,\n homeId,\n MaxEventTime,\n MinReading,\n MaxReading\n FROM iotdevicesample_GarageDoor_accumulated\n WHERE hour(MaxEventTime) = hour(current_timestamp());\n\n--DataXQuery--\nSELECT deviceId, deviceType, homeId, MaxEventTime, MinReading, MaxReading\nFROM GarageDoorAccumalator\nWITH UPSERT iotdevicesample_GarageDoor_accumulated;\n\n--DataXQuery--\nGarageDoorSliding = SELECT\n MaxEventTime AS EventTime,\n 'GarageDoorSliding' AS MetricName,\n MaxReading AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM iotdevicesample_GarageDoor_accumulated\n INNER JOIN GarageDoorStatus ON GarageDoorStatus.EventTime = iotdevicesample_GarageDoor_accumulated.MaxEventTime\n WHERE homeId = 150\n AND deviceType = 'GarageDoorLock';\n\n--DataXQuery--\nGarageDoorSlidingOpenInAnHour = SELECT COUNT(MaxReading) AS NumOfTimesGarageOpenFor5Mins,\n MAX(MaxEventTime) AS MaxEventTime\n FROM iotdevicesample_GarageDoor_accumulated\n WHERE homeId = 150\n AND deviceType = 'GarageDoorLock'\n AND MaxReading = 0;\n\n--DataXQuery--\nGarageOpenFor5MinsInAnHourAlert = SELECT \n MaxEventTime AS EventTime,\n 'GarageOpenFor5MinsInAnHourAlert' AS MetricName,\n 0 AS Metric,\n 'iotdevicesample' AS Product, \n CONCAT('No. of times garage door open for past 5 mins in current hour: ', NumOfTimesGarageOpenFor5Mins) AS Pivot1\n FROM GarageDoorSlidingOpenInAnHour;\n\nSimpleRulesTable = ProcessRules();\nAggregateRulesTable = ProcessAggregateRules(DataXProcessedInput);\nOUTPUT SimpleRulesTable TO myCosmosDB,myBlob;\nOUTPUT AggregateRulesTable TO myEventhub;\n\nOUTPUT HeaterStatus, WindowLockStatus, WindowLockSliding TO Metrics;\nOUTPUT WindowOpenWhileHeaterOnAlert TO Metrics;\nOUTPUT DoorLockStatus TO Metrics;\nOUTPUT DoorLockOpenBetween9pmAnd7amUtcAlert TO Metrics;\nOUTPUT GarageDoorStatus, GarageDoorSliding TO Metrics;\nOUTPUT GarageOpenFor5MinsInAnHourAlert TO Metrics;\n" ], "jobconfig": { "jobNumExecutors": "6", - "jobExecutorMemory": "8000" + "jobExecutorMemory": "8000", + "jobDatabricksAutoScale": false, + "jobDatabricksMinWorkers": null, + "jobDatabricksMaxWorkers": null } }, "outputs": [ @@ -564,8 +386,9 @@ } } ], + "batchList": "", "subscription": null }, - "properties": { "foo": "bar" }, - "jobNames": [ "localconfiggentest" ] + "properties": { "k1": "v1" }, + "jobNames": null } diff --git a/Services/DataX.Config/DataX.Config.Local.Test/Resource/flowStarted.json b/Services/DataX.Config/DataX.Config.Local.Test/Resource/flowStarted.json index 969df1276..ad58d6e3a 100644 --- a/Services/DataX.Config/DataX.Config.Local.Test/Resource/flowStarted.json +++ b/Services/DataX.Config/DataX.Config.Local.Test/Resource/flowStarted.json @@ -72,7 +72,8 @@ "timechart": true, "average": true, "speed": true - } + }, + "dynamicOffsetInMs": 300000 } }, { @@ -299,6 +300,7 @@ "gui": { "name": "localconfiggentest", "displayName": "localconfiggentest", + "databricksToken": null, "owner": "data-accelerator@microsoft.com", "input": { "mode": "streaming", @@ -313,7 +315,9 @@ "maxRate": "35000", "inputSchemaFile": "{\"type\":\"struct\",\"fields\":[{\"name\":\"deviceDetails\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"deviceId\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"deviceType\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"eventTime\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"homeId\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"status\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}", "showNormalizationSnippet": false, - "normalizationSnippet": "stringToTimestamp(Raw.deviceDetails.eventTime) AS eventTimeStamp\r\nRaw.*" + "normalizationSnippet": "stringToTimestamp(Raw.deviceDetails.eventTime) AS eventTimeStamp\r\nRaw.*", + "inputSubscriptionId": "", + "inputResourceGroup": "" }, "referenceData": [ { @@ -326,7 +330,8 @@ "header": true } } - ] + ], + "batch": null }, "process": { "timestampColumn": "eventTime", @@ -375,7 +380,10 @@ ], "jobconfig": { "jobNumExecutors": "6", - "jobExecutorMemory": "8000" + "jobExecutorMemory": "8000", + "jobDatabricksAutoScale": false, + "jobDatabricksMinWorkers": null, + "jobDatabricksMaxWorkers": null } }, "outputs": [ @@ -578,10 +586,11 @@ } } ], + "batchList": "", "subscription": null }, "properties": { - "foo": "bar" + "k1": "v1" }, "jobNames": [ "localconfiggentest" diff --git a/Services/DataX.Config/DataX.Config.Local.Test/Resource/localSparkJobTemplate.json b/Services/DataX.Config/DataX.Config.Local.Test/Resource/localSparkJobTemplate.json index 535c4ace8..026dd5d62 100644 --- a/Services/DataX.Config/DataX.Config.Local.Test/Resource/localSparkJobTemplate.json +++ b/Services/DataX.Config/DataX.Config.Local.Test/Resource/localSparkJobTemplate.json @@ -3,7 +3,7 @@ "cluster": "${sparkClusterName}", "options": { "name": "${sparkJobName}", - "file": "${localRoot}/bin/datax-host_2.3_2.11-1.1.0.jar", + "file": "${localRoot}/bin/datax-host_2.4_2.11-1.2.0.jar", "className": "datax.app.DataXDirectLocal", "args": [ "conf=${sparkJobConfigFilePath}", @@ -11,7 +11,7 @@ ], "jars": [ "${localRoot}/bin/datax-core.jar", - "${localRoot}/bin/datax-utility_2.3_2.11-1.1.0.jar", + "${localRoot}/bin/datax-utility_2.4_2.11-1.2.0.jar", "${localRoot}/bin/applicationinsights-core-2.2.1.jar", "${localRoot}/bin/azure-eventhubs-1.2.1.jar", "${localRoot}/bin/azure-eventhubs-spark_2.11-2.3.6.jar", diff --git a/Services/DataX.Config/DataX.Config.Local.Test/Resource/localconfiggentest-combined.txt b/Services/DataX.Config/DataX.Config.Local.Test/Resource/localconfiggentest-combined.txt index f6f04d797..27753f31b 100644 --- a/Services/DataX.Config/DataX.Config.Local.Test/Resource/localconfiggentest-combined.txt +++ b/Services/DataX.Config/DataX.Config.Local.Test/Resource/localconfiggentest-combined.txt @@ -268,7 +268,7 @@ --DataXQuery-- ar1_2_1 = SELECT - AVG(deviceDetails.deviceId) AS deviceDetails.deviceId_AVG, + AVG(deviceDetails.deviceId) AS deviceDetailsdeviceId_AVG, deviceDetails.deviceType, COUNT(*) AS Count FROM @@ -280,7 +280,7 @@ ar2_2_1 = SELECT *, IF( - deviceDetails.deviceId_AVG > 2, + deviceDetailsdeviceId_AVG > 2, MAP( 'ruleId', '5c85ede7-7a9a-4f64-992d-4c42dfd386c9', @@ -300,13 +300,13 @@ ar3_2_1 = SELECT MAP( 'deviceDetails.deviceType', - deviceDetails.deviceType + deviceType ) AS pivots, MAP( 'deviceDetails.deviceId', MAP( 'AVG', - deviceDetails.deviceId_AVG + deviceDetailsdeviceId_AVG ) ) AS aggs, Count AS count, diff --git a/Services/DataX.Config/DataX.Config.Local/DataX.Config.Local.csproj b/Services/DataX.Config/DataX.Config.Local/DataX.Config.Local.csproj index 3b8789ede..b4eca3224 100644 --- a/Services/DataX.Config/DataX.Config.Local/DataX.Config.Local.csproj +++ b/Services/DataX.Config/DataX.Config.Local/DataX.Config.Local.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -53,7 +53,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Config/DataX.Config.Local/KeyVaultClient.cs b/Services/DataX.Config/DataX.Config.Local/KeyVaultClient.cs index 8b74f34fc..fcad9ca98 100644 --- a/Services/DataX.Config/DataX.Config.Local/KeyVaultClient.cs +++ b/Services/DataX.Config/DataX.Config.Local/KeyVaultClient.cs @@ -2,6 +2,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using DataX.Config.ConfigDataModel; using System.Composition; using System.Threading.Tasks; @@ -25,9 +26,19 @@ public Task ResolveSecretUriAsync(string secretUri) return Task.FromResult(secretUri); } - public Task SaveSecretAsync(string keyvaultName, string secretName, string secretValue, bool hashSuffix = false) + public Task SaveSecretAsync(string keyvaultName, string secretName, string secretValue, string sparkType, bool hashSuffix = false) { return Task.FromResult(secretValue); } + + public Task SaveSecretAsync(string secretUri, string secretValue) + { + return Task.FromResult(secretValue); + } + + public string GetUriPrefix(string sparkType) + { + return (sparkType != null && sparkType == Constants.SparkTypeDataBricks) ? Constants.PrefixSecretScope : Constants.PrefixKeyVault; + } } } diff --git a/Services/DataX.Config/DataX.Config.Local/LocalRuntimeTimeStorage.cs b/Services/DataX.Config/DataX.Config.Local/LocalRuntimeTimeStorage.cs index b75c85b92..9e12492d9 100644 --- a/Services/DataX.Config/DataX.Config.Local/LocalRuntimeTimeStorage.cs +++ b/Services/DataX.Config/DataX.Config.Local/LocalRuntimeTimeStorage.cs @@ -2,6 +2,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using System; using System.Composition; using System.IO; using System.Threading.Tasks; @@ -26,6 +27,11 @@ public Task Delete(string destinationPath) return Task.FromResult(destinationPath); } + public Task DeleteAll(string destinationPath) + { + throw new NotImplementedException(); + } + private void DeleteLocal(string destinationPath) { var uri = new System.Uri(destinationPath); diff --git a/Services/DataX.Config/DataX.Config.Local/LocalSparkClientFactory.cs b/Services/DataX.Config/DataX.Config.Local/LocalSparkClientFactory.cs index 9b071e4b4..67be21049 100644 --- a/Services/DataX.Config/DataX.Config.Local/LocalSparkClientFactory.cs +++ b/Services/DataX.Config/DataX.Config.Local/LocalSparkClientFactory.cs @@ -17,7 +17,7 @@ public class LocalSparkClientFactory : ISparkJobClientFactory private ILogger _logger { get; } [ImportingConstructor] - public LocalSparkClientFactory(ILogger logger) + public LocalSparkClientFactory(ILogger logger) { this._logger = logger; } diff --git a/Services/DataX.Config/DataX.Config.Local/Resources/localSparkJobTemplate.json b/Services/DataX.Config/DataX.Config.Local/Resources/localSparkJobTemplate.json index a448b8a72..a22b18cdc 100644 --- a/Services/DataX.Config/DataX.Config.Local/Resources/localSparkJobTemplate.json +++ b/Services/DataX.Config/DataX.Config.Local/Resources/localSparkJobTemplate.json @@ -3,19 +3,19 @@ "cluster": "${sparkClusterName}", "options": { "name": "${sparkJobName}", - "file": "${localRoot}/bin/datax-host_2.3_2.11-1.1.0.jar", + "file": "${localRoot}/bin/datax-host_2.4_2.11-1.2.0.jar", "className": "datax.app.DirectLocalStreamingApp", "args": [ "conf=${sparkJobConfigFilePath}", "driverLogLevel=${sparkJobDriverLogLevel}" ], "jars": [ - "${localRoot}/bin/datax-core_2.3_2.11-1.1.0.jar", - "${localRoot}/bin/datax-utility_2.3_2.11-1.1.0.jar", + "${localRoot}/bin/datax-core_2.4_2.11-1.2.0.jar", + "${localRoot}/bin/datax-utility_2.4_2.11-1.2.0.jar", "${localRoot}/bin/applicationinsights-core-2.2.1.jar", "${localRoot}/bin/azure-eventhubs-1.2.1.jar", "${localRoot}/bin/azure-eventhubs-spark_2.11-2.3.6.jar", - "${localRoot}/bin/datax-keyvault_2.3_2.11-1.1.0-with-dependencies.jar", + "${localRoot}/bin/datax-keyvault_2.4_2.11-1.2.0-with-dependencies.jar", "${localRoot}/bin/proton-j-0.31.0.jar" ], diff --git a/Services/DataX.Config/DataX.Config.Storage/AzureBlobConfigStorage.cs b/Services/DataX.Config/DataX.Config.Storage/AzureBlobConfigStorage.cs index 33a8cb620..9393d6aa9 100644 --- a/Services/DataX.Config/DataX.Config.Storage/AzureBlobConfigStorage.cs +++ b/Services/DataX.Config/DataX.Config.Storage/AzureBlobConfigStorage.cs @@ -76,6 +76,22 @@ public async Task Delete(string destinationPath) await BlobUtility.DeleteBlob(connectionString, filePath); return destinationPath; } - + + public async Task DeleteAll(string destinationPath) + { + string connectionStringRef = Configuration[ConfigSettingName_RuntimeStorageConnectionString]; + string connectionString = await KeyVaultClient.ResolveSecretUriAsync(connectionStringRef); + var filePath = NormalizeFilePath(destinationPath); + if (Uri.TryCreate(destinationPath, UriKind.Absolute, out var uri)) + { + await BlobUtility.DeleteAllBlobsInAContainer(connectionString, uri.UserInfo, uri.PathAndQuery.TrimStart('/')); + } + else + { + throw new ArgumentException($"Malformed Uri for output:'{destinationPath}'"); + } + return destinationPath; + } + } } diff --git a/Services/DataX.Config/DataX.Config.Storage/DataX.Config.Storage.csproj b/Services/DataX.Config/DataX.Config.Storage/DataX.Config.Storage.csproj index 96d12d75b..ffdc086ac 100644 --- a/Services/DataX.Config/DataX.Config.Storage/DataX.Config.Storage.csproj +++ b/Services/DataX.Config/DataX.Config.Storage/DataX.Config.Storage.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -41,7 +41,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Config/DataX.Config.Test.Utility/DataX.Config.Test.Utility.csproj b/Services/DataX.Config/DataX.Config.Test.Utility/DataX.Config.Test.Utility.csproj index 81771ff70..2623bffb3 100644 --- a/Services/DataX.Config/DataX.Config.Test.Utility/DataX.Config.Test.Utility.csproj +++ b/Services/DataX.Config/DataX.Config.Test.Utility/DataX.Config.Test.Utility.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -21,7 +21,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Config/DataX.Config.Test/CompositionTest.cs b/Services/DataX.Config/DataX.Config.Test/CompositionTest.cs index d1ab531c5..a41582139 100644 --- a/Services/DataX.Config/DataX.Config.Test/CompositionTest.cs +++ b/Services/DataX.Config/DataX.Config.Test/CompositionTest.cs @@ -9,6 +9,7 @@ using System.Composition; using System.Composition.Hosting; using System.Reflection; +using DataX.Config.Utility; namespace DataX.Config.Test { @@ -21,7 +22,8 @@ public static void Initialize(TestContext tc) var conf = new ContainerConfiguration() .WithAssembly(typeof(ConfigGenConfiguration).Assembly) .WithAssembly(typeof(MockBase).Assembly) - .WithAssembly(Assembly.GetExecutingAssembly()); + .WithAssembly(Assembly.GetExecutingAssembly()) + .WithProvider(new LoggerAndInstanceExportDescriptorProvider(null, new LoggerFactory())); CompositionHost = conf.CreateContainer(); } diff --git a/Services/DataX.Config/DataX.Config.Test/CustomRuntimeConfigGenerationTest.cs b/Services/DataX.Config/DataX.Config.Test/CustomRuntimeConfigGenerationTest.cs new file mode 100644 index 000000000..fa616003f --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/CustomRuntimeConfigGenerationTest.cs @@ -0,0 +1,146 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Config.ConfigDataModel; +using DataX.Config.ConfigGeneration.Processor; +using DataX.Config.Test.Extension; +using DataX.Config.Test.Mock; +using DataX.Config.Test.Utility.Mock; +using DataX.Config.Utility; +using Microsoft.VisualStudio.TestTools.UnitTesting; +using System; +using System.Collections.Generic; +using System.Composition; +using System.Composition.Hosting; +using System.IO; +using System.Linq; +using System.Reflection; +using System.Threading.Tasks; + +namespace DataX.Config.Test +{ + [TestClass] + public class RuntimeConfigGenerationTestCustom + { + [ClassInitialize] + public static void Initialize(TestContext tc) + { + InitialConfiguration.Set(Constants.ConfigSettingName_ClusterName, "somecluster"); + InitialConfiguration.Set(Constants.ConfigSettingName_ServiceKeyVaultName, "someservicekeyvault"); + InitialConfiguration.Set(Constants.ConfigSettingName_RuntimeKeyVaultName, "somekeyvault"); + InitialConfiguration.Set(Constants.ConfigSettingName_MetricEventHubConnectionKey, "metric-eventhubconnectionstring"); + + var conf = new ContainerConfiguration() + .WithAssembly(typeof(ConfigGenConfiguration).Assembly) + .WithAssembly(typeof(MockBase).Assembly) + .WithAssembly(Assembly.GetExecutingAssembly()) + .WithProvider(new LoggerAndInstanceExportDescriptorProvider(null, new LoggerFactory())); + + CompositionHost = conf.CreateContainer(); + } + + [ClassCleanup] + public static void Cleanup() + { + if (CompositionHost != null) + { + CompositionHost.Dispose(); + CompositionHost = null; + } + + InitialConfiguration.Clear(); + } + + public RuntimeConfigGenerationTestCustom() + { + CompositionHost.SatisfyImports(this); + } + + private static CompositionHost CompositionHost { get; set; } + + [Import] + private RuntimeConfigGeneration RuntimeConfigGeneration { get; set; } + + [Import] + private DesignTimeStorage DesignTimeStorage { get; set; } + + [Import] + private RuntimeStorage RuntimeStorage { get; set; } + + [Import] + private ICommonDataManager CommonData { get; set; } + + [Import] + private ConfigurationProvider ConfigurationProvider { get; set; } + + [Shared] + [Export(typeof(IFlowDeploymentProcessor))] + private class VerifyJsonConfigGenerated : ProcessorBase + { + public override int GetOrder() + { + // set an order number to be placed right after the GenerateJobConfig processor + return 601; + } + + public override async Task Process(FlowDeploymentSession flowToDeploy) + { + return await Task.FromResult("done"); + } + } + + [TestMethod] + public async Task EndToEndGenerationCustom() + { + var flowName = "customconfiggentest"; + + var testingConfig = await File.ReadAllTextAsync(@"Resource\customflow.json"); + await DesignTimeStorage.SaveByName(flowName, testingConfig, FlowDataManager.DataCollectionName); + + await CommonData.Add("defaultFlowConfig", @"Resource\customflow.json"); + await CommonData.Add("flattener", @"Resource\customFlattenerConfig.json"); + await CommonData.Add("defaultJobTemplate", @"Resource\sparkJobTemplate.json"); + + var result = await this.RuntimeConfigGeneration.GenerateRuntimeConfigs(flowName); + + var runtimeConfigFolder = result.Properties?.GetValueOrDefault(PrepareJobConfigVariables.ResultPropertyName_RuntimeConfigFolder, null); + + Assert.IsTrue(result.IsSuccess); + Assert.AreEqual(expected: 2, actual: RuntimeStorage.Cache.Count); + + var jobConfigDestinationFolder = runtimeConfigFolder?.ToString().Split("Generation_").First(); + + // Verify output configuration is expected + var actualConf = PropertiesDictionary.From(this.RuntimeStorage.Cache[ResourcePathUtil.Combine(runtimeConfigFolder.ToString(), "customconfiggentest1.conf")]); + var expectedConf = PropertiesDictionary.From(await File.ReadAllTextAsync(@"Resource\customJobConfig1.conf")); + var matches = PropertiesDictionary.Match(expectedConf, actualConf).ToList(); + foreach (var match in matches) + { + Console.WriteLine($"prop:{match.Item1 ?? "null"}, expected:<{match.Item2 ?? "null"}>, actual:<{match.Item3 ?? "null"}>"); + } + + foreach (var match in matches) + { + Assert.AreEqual(expected: match.Item2, actual: match.Item3, message: $"property:{match.Item1}"); + } + + //verify second job conf is generated as expected + flowName = "customconfiggentest2"; + actualConf = PropertiesDictionary.From(this.RuntimeStorage.Cache[ResourcePathUtil.Combine(runtimeConfigFolder.ToString(), "customconfiggentest2.conf")]); + expectedConf = PropertiesDictionary.From(await File.ReadAllTextAsync(@"Resource\customJobConfig2.conf")); + matches = PropertiesDictionary.Match(expectedConf, actualConf).ToList(); + foreach (var match in matches) + { + Console.WriteLine($"prop:{match.Item1 ?? "null"}, expected:<{match.Item2 ?? "null"}>, actual:<{match.Item3 ?? "null"}>"); + } + + foreach (var match in matches) + { + Assert.AreEqual(expected: match.Item2, actual: match.Item3, message: $"property:{match.Item1}"); + } + + Cleanup(); + } + } +} diff --git a/Services/DataX.Config/DataX.Config.Test/DataX.Config.Test.csproj b/Services/DataX.Config/DataX.Config.Test/DataX.Config.Test.csproj index f689bee7b..087c305fa 100644 --- a/Services/DataX.Config/DataX.Config.Test/DataX.Config.Test.csproj +++ b/Services/DataX.Config/DataX.Config.Test/DataX.Config.Test.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 false @@ -37,12 +37,34 @@ + + + PreserveNewest + + + PreserveNewest + + + PreserveNewest + + + PreserveNewest + + + PreserveNewest + + + PreserveNewest + + + PreserveNewest + Always @@ -55,9 +77,27 @@ PreserveNewest + + PreserveNewest + + + PreserveNewest + + + Always + Always + + PreserveNewest + + + PreserveNewest + + + PreserveNewest + PreserveNewest @@ -92,7 +132,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Config/DataX.Config.Test/DatabricksRuntimeConfigGenerationTest.cs b/Services/DataX.Config/DataX.Config.Test/DatabricksRuntimeConfigGenerationTest.cs new file mode 100644 index 000000000..d6909b1a3 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/DatabricksRuntimeConfigGenerationTest.cs @@ -0,0 +1,170 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Microsoft.VisualStudio.TestTools.UnitTesting; +using DataX.Config.ConfigDataModel; +using DataX.Config.ConfigGeneration.Processor; +using DataX.Config.Test.Extension; +using DataX.Config.Test.Mock; +using DataX.Config.Test.Utility.Mock; +using DataX.Config.Utility; +using System; +using System.Collections.Generic; +using System.Composition; +using System.Composition.Hosting; +using System.IO; +using System.Linq; +using System.Reflection; +using System.Threading.Tasks; + +namespace DataX.Config.Test +{ + [TestClass] + public class DatabricksRuntimeConfigGenerationTest + { + [ClassInitialize] + public static void Initialize(TestContext tc) + { + InitialConfiguration.Set(Constants.ConfigSettingName_ClusterName, "somecluster"); + InitialConfiguration.Set(Constants.ConfigSettingName_ServiceKeyVaultName, "someservicekeyvault"); + InitialConfiguration.Set(Constants.ConfigSettingName_RuntimeKeyVaultName, "somekeyvault"); + InitialConfiguration.Set(Constants.ConfigSettingName_MetricEventHubConnectionKey, "metric-eventhubconnectionstring"); + InitialConfiguration.Set(Constants.ConfigSettingName_SparkType, "databricks"); + + var conf = new ContainerConfiguration() + .WithAssembly(typeof(ConfigGenConfiguration).Assembly) + .WithAssembly(typeof(MockBase).Assembly) + .WithAssembly(Assembly.GetExecutingAssembly()) + .WithProvider(new LoggerAndInstanceExportDescriptorProvider(null, new LoggerFactory())); + + CompositionHost = conf.CreateContainer(); + } + + [ClassCleanup] + public static void Cleanup() + { + if (CompositionHost != null) + { + CompositionHost.Dispose(); + CompositionHost = null; + } + + InitialConfiguration.Clear(); + } + + public DatabricksRuntimeConfigGenerationTest() + { + CompositionHost.SatisfyImports(this); + } + + private static CompositionHost CompositionHost { get; set; } + + [Import] + private RuntimeConfigGeneration RuntimeConfigGeneration { get; set; } + + [Import] + private DesignTimeStorage DesignTimeStorage { get; set; } + + [Import] + private RuntimeStorage RuntimeStorage { get; set; } + + [Import] + private ICommonDataManager CommonData { get; set; } + + [Import] + private ConfigurationProvider ConfigurationProvider { get; set; } + + [Shared] + [Export(typeof(IFlowDeploymentProcessor))] + private class DatabricksVerifyJsonConfigGenerated : ProcessorBase + { + public override int GetOrder() + { + // set an order number to be placed right after the GenerateJobConfig processor + return 601; + } + + public override async Task Process(FlowDeploymentSession flowToDeploy) + { + if(flowToDeploy.Name == "dbconfiggentest") + { + var expectedConfigContent = await File.ReadAllTextAsync(@"Resource\databricksJobConfig.json"); + var expectedJson = JsonConfig.From(expectedConfigContent); + var actualContentContent = flowToDeploy.GetJobs().First().GetTokenString(GenerateJobConfig.TokenName_JobConfigContent); + var actualJson = JsonConfig.From(actualContentContent); + + foreach (var match in JsonConfig.Match(expectedJson, actualJson)) + { + Assert.AreEqual(expected: match.Item2, actual: match.Item3, message: $"path:{match.Item1}"); + } + } + + return "done"; + } + } + + [TestMethod] + public async Task DatabricksTestEndToEndGeneration() + { + var flowName = "dbconfiggentest"; + + var testingConfig = await File.ReadAllTextAsync(@"Resource\databricksFlowSaved.json"); + await DesignTimeStorage.SaveByName(flowName, testingConfig, FlowDataManager.DataCollectionName); + + await CommonData.Add("defaultJobTemplate", @"Resource\sparkJobTemplate.json"); + await CommonData.Add(ConfigFlattenerManager.DefaultConfigName, @"Resource\flattenerConfig.json"); + await CommonData.Add(FlowDataManager.CommonDataName_DefaultFlowConfig, @"Resource\flowDefault.json"); + + var result = await this.RuntimeConfigGeneration.GenerateRuntimeConfigs(flowName); + + var runtimeConfigFolder = result.Properties?.GetValueOrDefault(PrepareJobConfigVariables.ResultPropertyName_RuntimeConfigFolder, null); + + Assert.IsTrue(result.IsSuccess); + Assert.AreEqual(expected: 4, actual: RuntimeStorage.Cache.Count); + + // verify output schema file is expected + var expectedSchema = JsonConfig.From(await File.ReadAllTextAsync(@"Resource\schema.json")); + var actualSchema = JsonConfig.From(RuntimeStorage.Cache[ResourcePathUtil.Combine(runtimeConfigFolder.ToString(), "inputschema.json")]); + foreach (var match in JsonConfig.Match(expectedSchema, actualSchema)) + { + Assert.AreEqual(expected: match.Item2, actual: match.Item3, message: $"path:{match.Item1}"); + } + + // verify output projection file is expected + var expectedProjection = await File.ReadAllTextAsync(@"Resource\projection.txt"); + var actualProjection = RuntimeStorage.Cache[ResourcePathUtil.Combine(runtimeConfigFolder.ToString(), "projection.txt")]; + Assert.AreEqual(expected: expectedProjection, actual: actualProjection); + + // verify transform file is exepcted + var expectedTransform = await File.ReadAllTextAsync(@"Resource\configgentest-combined.txt"); + var actualTransform = RuntimeStorage.Cache[ResourcePathUtil.Combine(runtimeConfigFolder.ToString(), "dbconfiggentest-combined.txt")]; + Assert.AreEqual(expected: expectedTransform, actual: actualTransform); + + // Verify output configuration is expected + var actualConf = PropertiesDictionary.From(this.RuntimeStorage.Cache[ResourcePathUtil.Combine(runtimeConfigFolder.ToString(), $"{flowName}.conf")]); + var expectedConf = PropertiesDictionary.From(await File.ReadAllTextAsync(@"Resource\databricksJobConfig.conf")); + var matches = PropertiesDictionary.Match(expectedConf, actualConf).ToList(); + foreach (var match in matches) + { + Console.WriteLine($"prop:{match.Item1 ?? "null"}, expected:<{match.Item2 ?? "null"}>, actual:<{match.Item3 ?? "null"}>"); + } + + foreach (var match in matches) + { + Assert.AreEqual(expected: match.Item2, actual: match.Item3, message: $"property:{match.Item1}"); + } + + // Verify metrics + var expectedConfig = JsonConfig.From(await File.ReadAllTextAsync(@"Resource\databricksFlowStarted.json")); + var actualConfig = JsonConfig.From(await this.DesignTimeStorage.GetByName(flowName, FlowDataManager.DataCollectionName)); + + foreach (var match in JsonConfig.Match(expectedConfig, actualConfig)) + { + Assert.AreEqual(expected: match.Item2, actual: match.Item3, message: $"path:{match.Item1}"); + } + + Cleanup(); + } + } +} diff --git a/Services/DataX.Config/DataX.Config.Test/FlowOperationTest.cs b/Services/DataX.Config/DataX.Config.Test/FlowOperationTest.cs index 99faf5cba..38cbfd437 100644 --- a/Services/DataX.Config/DataX.Config.Test/FlowOperationTest.cs +++ b/Services/DataX.Config/DataX.Config.Test/FlowOperationTest.cs @@ -17,6 +17,7 @@ using System.Reflection; using System.Text; using System.Threading.Tasks; +using DataX.Config.Utility; namespace DataX.Config.Test { @@ -31,7 +32,8 @@ public static void Initialize(TestContext tc) var conf = new ContainerConfiguration() .WithAssembly(typeof(ConfigGenConfiguration).Assembly) .WithAssembly(typeof(MockBase).Assembly) - .WithAssembly(Assembly.GetExecutingAssembly()); + .WithAssembly(Assembly.GetExecutingAssembly()) + .WithProvider(new LoggerAndInstanceExportDescriptorProvider(null, new LoggerFactory())); CompositionHost = conf.CreateContainer(); } @@ -78,6 +80,8 @@ public async Task TestConfigSaved() { Assert.AreEqual(expected: match.Item2, actual: match.Item3, message: $"path:{match.Item1}"); } + + Cleanup(); } } } diff --git a/Services/DataX.Config/DataX.Config.Test/LoggerFactory.cs b/Services/DataX.Config/DataX.Config.Test/LoggerFactory.cs new file mode 100644 index 000000000..39befc39b --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/LoggerFactory.cs @@ -0,0 +1,28 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Config.Test.Utility.Mock; +using Microsoft.Extensions.Logging; + +namespace DataX.Config.Test +{ + internal class LoggerFactory : ILoggerFactory + { + public void AddProvider(ILoggerProvider provider) + { + throw new System.NotImplementedException(); + } + + public ILogger CreateLogger(string categoryName) + { + var result = new CacheLogger(); + return result; + } + + public void Dispose() + { + throw new System.NotImplementedException(); + } + } +} diff --git a/Services/DataX.Config/DataX.Config.Test/Mock/KeyVaultClient.cs b/Services/DataX.Config/DataX.Config.Test/Mock/KeyVaultClient.cs index 1579a35e0..13bdb0841 100644 --- a/Services/DataX.Config/DataX.Config.Test/Mock/KeyVaultClient.cs +++ b/Services/DataX.Config/DataX.Config.Test/Mock/KeyVaultClient.cs @@ -2,6 +2,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using DataX.Config.ConfigDataModel; using DataX.Config.KeyVault; using System; using System.Collections.Generic; @@ -28,15 +29,30 @@ public async Task GetSecretFromKeyVaultAsync(string keyvaultName, string public async Task ResolveSecretUriAsync(string secretUri) { - await Task.Yield(); + if (secretUri == "keyvault://somekeyvault/configgenbatchtest-input-0-inputConnection") + { + return await Task.FromResult(@"DefaultEndpointsProtocol=https;AccountName=testaccount;AccountKey=testkey;EndpointSuffix=core.windows.net"); + } return secretUri; } - public async Task SaveSecretAsync(string keyvaultName, string secretName, string secretValue, bool hashSuffix = false) + public async Task SaveSecretAsync(string keyvaultName, string secretName, string secretValue, string sparkType, bool hashSuffix = false) { + var uriPrefix = GetUriPrefix(sparkType); var finalSecretName = hashSuffix ? (secretName + "-" + HashGenerator.GetHashCode(secretValue)) : secretName; await Task.Yield(); - return $"keyvault://{keyvaultName}/{finalSecretName}"; + return $"{uriPrefix}://{keyvaultName}/{finalSecretName}"; + } + + public async Task SaveSecretAsync(string secretUri, string secretValue) + { + await Task.Yield(); + return secretUri; + } + + public string GetUriPrefix(string sparkType) + { + return (sparkType != null && sparkType == Constants.SparkTypeDataBricks) ? Constants.PrefixSecretScope : Constants.PrefixKeyVault; } } } diff --git a/Services/DataX.Config/DataX.Config.Test/Mock/RuntimeStorage.cs b/Services/DataX.Config/DataX.Config.Test/Mock/RuntimeStorage.cs index 37fc7ebfb..4520378d8 100644 --- a/Services/DataX.Config/DataX.Config.Test/Mock/RuntimeStorage.cs +++ b/Services/DataX.Config/DataX.Config.Test/Mock/RuntimeStorage.cs @@ -41,5 +41,9 @@ public Task Delete(string destinationPath) return Task.FromResult(destinationPath); } + public Task DeleteAll(string destinationPath) + { + throw new NotImplementedException(); + } } } diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/batchFlow.json b/Services/DataX.Config/DataX.Config.Test/Resource/batchFlow.json new file mode 100644 index 000000000..eef4fe0a2 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/Resource/batchFlow.json @@ -0,0 +1,197 @@ +{ + "name": "configgenbatchtest", + "icon": "/img/iot.png", + "displayName": "configgenbatchtest", + "properties": { + "foo": "bar" + }, + "gui": { + "name": "configgenbatchtest", + "displayName": "configgenbatchtest", + "owner": "data-accelerator@microsoft.com", + "input": { + "mode": "batching", + "type": "blob", + "properties": { + "inputEventhubName": "", + "inputEventhubConnection": "", + "windowDuration": "30", + "timestampColumn": "", + "watermarkValue": "0", + "watermarkUnit": "second", + "maxRate": "1000", + "inputSchemaFile": "{\"type\":\"struct\",\"fields\":[{\"name\":\"deviceDetails\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"deviceType\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"id\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"deviceId\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"homeId\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"status\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}", + "showNormalizationSnippet": false, + "normalizationSnippet": "Raw.*", + "inputSubscriptionId": "", + "inputResourceGroup": "" + }, + "referenceData": [], + "batch": [ + { + "type": "blob", + "properties": { + "connection": "keyvault://somekeyvault/configgenbatchtest-input-0-inputConnection", + "path": "keyvault://somekeyvault/configgenbatchtest-input-0-inputPath", + "formatType": "json", + "compressionType": "none" + } + } + ] + }, + "process": { + "timestampColumn": "", + "watermark": "0 second", + "functions": [], + "queries": [ + "--DataXQuery--\nevents = SELECT\nCOUNT(*) AS eventCount\nFROM \nDataXProcessedInput;\n\n--DataXQuery--\nt1 = SELECT\n*\nFROM \nDataXProcessedInput;\n\nmetricExample = CreateMetric(events, eventCount);\n\nOUTPUT metricExample TO Metrics;\nOUTPUT DataXProcessedInput TO myBlob;" + ], + "jobconfig": { + "jobNumExecutors": "4", + "jobExecutorMemory": "1000" + } + }, + "outputs": [ + { + "id": "Metrics", + "type": "metric", + "properties": {}, + "typeDisplay": "Metrics" + } + ], + "outputTemplates": [], + "rules": [], + "batchList": [ + { + "id": "r", + "type": "recurring", + "properties": { + "interval": "1", + "intervalType": "day", + "delay": "0", + "delayType": "day", + "window": "3", + "windowType": "day", + "startTime": "${startTime}", + "endTime": "${endTime}", + "lastProcessedTime": "" + }, + "disabled": false + }, + { + "id": "o", + "type": "oneTime", + "properties": { + "interval": "1", + "intervalType": "day", + "delay": "0", + "delayType": "day", + "window": "1", + "windowType": "day", + "startTime": "${startTime}", + "endTime": "${endTime}", + "lastProcessedTime": "" + }, + "disabled": false + } + ], + "subscription": "" + }, + "commonProcessor": { + "sparkJobConfigFolder": "file:///datax/runtime/testjobs/", + "template": { + "name": "${name}", + "input": { + "blob": "${inputBatching}", + "blobSchemaFile": "${inputSchemaFilePath}", + "referenceData": "${inputReferenceData}" + }, + "process": { + "metric": { + "eventhub": "keyvault://${sparkKeyVaultName}/${metricEventHubConnectionStringKey}" + }, + "timestampColumn": "${processTimestampColumn}", + "watermark": "${processWatermark}", + "jarUDAFs": "${processJarUDAFs}", + "jarUDFs": "${processJarUDFs}", + "azureFunctions": "${processAzureFunctions}", + "projections": "${processProjections}", + "timeWindows": "${processTimeWindows}", + "transform": "${processTransforms}", + "appendEventTags": {}, + "accumulationTables": "${processStateTables}" + }, + "outputs": "${outputs}" + }, + "sparkJobTemplateRef": "defaultJobTemplate", + "jobCommonTokens": { + "jobName": "${name}", + "sparkJobName": "DataXBatch-${name}", + "sparkJobDriverLogLevel": "WARN", + "sparkJobExecutorLogLevel": "INFO", + "sparkJobNumExecutors": "${guiSparkJobNumExecutors}", + "sparkJobExecutorMemory": "${guiSparkJobExecutorMemory}", + "processedSchemaPath": null + }, + "jobs": [ + { + "partitionJobNumber": "1" + } + ] + }, + "metrics": { + "sources": [ + { + "name": "events", + "input": { + "type": "MetricApi", + "metricKeys": [ + "DATAX-configgenbatchtest:Input_DataXProcessedInput_Events_Count" + ] + }, + "output": { + "type": "SumWithTimeChart", + "data": { + "sum": true, + "timechart": true, + "average": true, + "speed": true + } + } + } + ], + "widgets": [ + { + "name": "eventsChart", + "displayName": "Events/Second", + "data": "events_timechart", + "position": "TimeCharts", + "type": "StackAreaChart" + }, + { + "name": "totalEvents", + "displayName": "Events Ingested Today", + "data": "events_sum", + "formatter": "longint", + "position": "FirstRow", + "type": "SimpleBox" + }, + { + "name": "averageEvents", + "displayName": "Avg. Events/Minute", + "data": "events_average", + "formatter": "longint", + "position": "FirstRow", + "type": "SimpleBox" + } + ], + "initParameters": { + "widgetSets": [ + "direct" + ], + "jobNames": { + "type": "getCPSparkJobNames" + } + } + } +} \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/batchFlowDefault.json b/Services/DataX.Config/DataX.Config.Test/Resource/batchFlowDefault.json new file mode 100644 index 000000000..1a401905c --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/Resource/batchFlowDefault.json @@ -0,0 +1,105 @@ +{ + "name": "###REQUIRED###", + "icon": "/img/iot.png", + "displayName": "###REQUIRED###", + "properties": { + "foo": "bar" + }, + "commonProcessor": { + "sparkJobConfigFolder": "file:///datax/runtime/testjobs/", + "template": { + "name": "${name}", + "input": { + "blob": "${inputBatching}", + "blobSchemaFile": "${inputSchemaFilePath}", + "referenceData": "${inputReferenceData}" + }, + "process": { + "metric": { + "eventhub": "keyvault://somekeyvault/metric-eventhubconnectionstring" + }, + "timestampColumn": "${processTimestampColumn}", + "watermark": "${processWatermark}", + "jarUDAFs": "${processJarUDAFs}", + "jarUDFs": "${processJarUDFs}", + "azureFunctions": "${processAzureFunctions}", + "projections": "${processProjections}", + "timeWindows": "${processTimeWindows}", + "transform": "${processTransforms}", + "appendEventTags": {}, + "accumulationTables": "${processStateTables}" + }, + "outputs": "${outputs}" + }, + "sparkJobTemplateRef": "defaultJobTemplate", + "jobCommonTokens": { + "jobName": "${name}", + "sparkJobName": "DataXDirect-${name}", + "sparkJobDriverLogLevel": "WARN", + "sparkJobExecutorLogLevel": "INFO", + "sparkJobNumExecutors": "${guiSparkJobNumExecutors}", + "sparkJobExecutorMemory": "${guiSparkJobExecutorMemory}", + "processedSchemaPath": null + }, + "jobs": [ + { + "partitionJobNumber": "1" + } + ] + }, + "metrics": { + "sources": [ + { + "name": "events", + "input": { + "type": "MetricApi", + "metricKeys": [ + "DATAX-${name}:Input_DataXProcessedInput_Events_Count" + ] + }, + "output": { + "type": "SumWithTimeChart", + "data": { + "sum": true, + "timechart": true, + "average": true, + "speed": true + } + } + } + ], + "widgets": [ + { + "name": "eventsChart", + "displayName": "Events/Second", + "data": "events_timechart", + "position": "TimeCharts", + "type": "StackAreaChart" + }, + { + "name": "totalEvents", + "displayName": "Events Ingested Today", + "data": "events_sum", + "formatter": "longint", + "type": "SimpleBox", + "position": "FirstRow" + }, + { + "name": "averageEvents", + "displayName": "Avg. Events/Minute", + "data": "events_average", + "formatter": "longint", + "type": "SimpleBox", + "position": "FirstRow" + } + ], + "initParameters": { + "widgetSets": [ + "direct" + ], + "jobNames": { + "type": "getCPSparkJobNames" + } + } + } +} \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/batchJobConfig.conf b/Services/DataX.Config/DataX.Config.Test/Resource/batchJobConfig.conf new file mode 100644 index 000000000..6eef3524e --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/Resource/batchJobConfig.conf @@ -0,0 +1,31 @@ +# Configuration settings for the job +# 2018-12-31 - created +# ============================================================================ + +# Name of the job +# ---------------------------------------------------------------------------- +datax.job.name=configgenbatchtest-${processTime} + +# Input settings +# ---------------------------------------------------------------------------- +datax.job.input.default.blobschemafile=keyvault://somekeyvault/configgenbatchtest-inputschemafile +datax.job.input.default.blob.testaccount.path=keyvault://somekeyvault/configgenbatchtest-input-0-inputPath + +datax.job.input.default.blob.testaccount.format=json +datax.job.input.default.blob.testaccount.compressiontype=none +datax.job.input.default.blob.testaccount.processstarttime=${startTime} +datax.job.input.default.blob.testaccount.processendtime=${endTime} +datax.job.input.default.blob.testaccount.partitionincrement=1 + +# Process settings +# ---------------------------------------------------------------------------- +datax.job.process.metric.eventhub=keyvault://somekeyvault/metric-eventhubconnectionstring +datax.job.process.projection=keyvault://somekeyvault/configgenbatchtest-projectionfile +datax.job.process.transform=keyvault://somekeyvault/configgenbatchtest-transform +datax.job.process.timestampcolumn= +datax.job.process.watermark=0 second + +# Output settings +# ---------------------------------------------------------------------------- +datax.job.output.metricExample.eventhub.connectionstring=keyvault://somekeyvault/metric-eventhubconnectionstring +datax.job.output.metricExample.eventhub.compressiontype=none \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/batchSparkJobTemplate.json b/Services/DataX.Config/DataX.Config.Test/Resource/batchSparkJobTemplate.json new file mode 100644 index 000000000..a934e7dc5 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/Resource/batchSparkJobTemplate.json @@ -0,0 +1,9 @@ +{ + "name": "${sparkJobName}", + "cluster": "test", + "options": { + }, + "batch": null, + "state": "Idle", + "note": null +} \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/customFlattenerConfig.json b/Services/DataX.Config/DataX.Config.Test/Resource/customFlattenerConfig.json new file mode 100644 index 000000000..62e0df5b1 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/Resource/customFlattenerConfig.json @@ -0,0 +1,307 @@ +{ + "type": "object", + "namespace": "datax.job", + "fields": { + "name": "name", + "input": { + "type": "object", + "namespace": "input.default", + "fields": { + "blobSchemaFile": "blobschemafile", + "sourceIdRegex": "sourceidregex", + "blobPathRegex": "blobpathregex", + "fileTimeRegex": "filetimeregex", + "fileTimeFormat": "filetimeformat", + "eventNamePath": "eventnamepath", + "eventhub": { + "type": "object", + "namespace": "eventhub", + "fields": { + "connectionString": "connectionstring", + "consumerGroup": "consumergroup", + "checkpointDir": "checkpointdir", + "checkpointInterval": "checkpointinterval", + "maxRate": "maxrate", + "startEnqueueTime": "startenqueuetime", + "partitionCount": "partitioncount" + } + }, + "streaming": { + "type": "object", + "namespace": "streaming", + "fields": { + "checkpointDir": "checkpointdir", + "intervalInSeconds": "intervalinseconds" + } + }, + "sources": { + "type": "map", + "namespace": "source", + "fields": { + "target": "target", + "catalogPrefix": "catalogprefix" + } + }, + "referenceData": { + "type": "array", + "namespace": "referencedata", + "element": { + "type": "scopedObject", + "namespaceField": "name", + "fields": { + "path": "path", + "format": "format", + "header": "header", + "delimiter": "delimiter" + } + } + } + } + }, + "process": { + "type": "object", + "namespace": "process", + "fields": { + "metric": { + "type": "object", + "namespace": "metric", + "fields": { + "eventhub": "eventhub" + } + }, + "projections": { + "type": "stringList", + "namespace": "projection" + }, + "transform": "transform", + "timestampColumn": "timestampcolumn", + "watermark": "watermark", + "preprojection": "preprojection", + "udf": { + "type": "object", + "namespace": "udf", + "fields": { + "tagWorkload": "tagWorkload" + } + }, + "timeWindows": { + "type": "array", + "namespace": "timewindow", + "element": { + "type": "scopedObject", + "namespaceField": "name", + "fields": { "windowDuration": "windowduration" } + } + }, + "jarUDFs": { + "type": "array", + "namespace": "jar.udf", + "element": { + "type": "scopedObject", + "namespaceField": "name", + "fields": { + "class": "class", + "path": "path", + "libs": { + "type": "stringList", + "namespace": "libs" + } + } + } + }, + "jarUDAFs": { + "type": "array", + "namespace": "jar.udaf", + "element": { + "type": "scopedObject", + "namespaceField": "name", + "fields": { + "class": "class", + "path": "path", + "libs": { + "type": "stringList", + "namespace": "libs" + } + } + } + }, + "accumulationTables": { + "type": "array", + "namespace": "statetable", + "element": { + "type": "scopedObject", + "namespaceField": "name", + "fields": { + "schema": "schema", + "location": "location" + } + } + }, + "azureFunctions": { + "type": "array", + "namespace": "azurefunction", + "element": { + "type": "scopedObject", + "namespaceField": "name", + "fields": { + "serviceEndpoint": "serviceendpoint", + "api": "api", + "code": "code", + "methodType": "methodtype", + "params": { + "type": "stringList", + "namespace": "params" + } + } + } + } + } + }, + "output": { + "type": "scopedObject", + "namespace": "output", + "namespaceField": "name", + "fields": { + "processedSchemaPath": "processedschemapath", + "blob": { + "type": "object", + "namespace": "blob", + "fields": { + "groupEvaluation": "groupevaluation", + "compressionType": { + "type": "excludeDefaultValue", + "namespace": "compressiontype", + "defaultValue": "gzip" + }, + "format": { + "type": "excludeDefaultValue", + "namespace": "format", + "defaultValue": "json" + }, + "groups": { + "type": "map", + "namespace": "group", + "fields": { "folder": "folder" } + } + } + }, + "eventhub": { + "type": "object", + "namespace": "eventhub", + "fields": { + "connectionStringRef": "connectionstring", + "compressionType": { + "type": "excludeDefaultValue", + "namespace": "compressiontype", + "defaultValue": "gzip" + }, + "format": { + "type": "excludeDefaultValue", + "namespace": "format", + "defaultValue": "json" + }, + "appendProperties": { + "type": "mapProps", + "namespace": "appendproperty" + } + } + }, + "cosmosdb": { + "type": "object", + "namespace": "cosmosdb", + "fields": { + "connectionStringRef": "connectionstring", + "database": "database", + "collection": "collection" + } + }, + "httpPost": { + "type": "object", + "namespace": "httppost", + "fields": { + "endpoint": "endpoint", + "filter": "filter", + "appendHeaders": { + "type": "mapProps", + "namespace": "header" + } + } + } + } + }, + "outputs": { + "type": "array", + "element": { + "type": "scopedObject", + "namespace": "output", + "namespaceField": "name", + "fields": { + "blob": { + "type": "object", + "namespace": "blob", + "fields": { + "groupEvaluation": "groupevaluation", + "compressionType": { + "type": "excludeDefaultValue", + "namespace": "compressiontype", + "defaultValue": "gzip" + }, + "format": { + "type": "excludeDefaultValue", + "namespace": "format", + "defaultValue": "json" + }, + "groups": { + "type": "map", + "namespace": "group", + "fields": { "folder": "folder" } + } + } + }, + "eventhub": { + "type": "object", + "namespace": "eventhub", + "fields": { + "connectionStringRef": "connectionstring", + "compressionType": { + "type": "excludeDefaultValue", + "namespace": "compressiontype", + "defaultValue": "gzip" + }, + "format": { + "type": "excludeDefaultValue", + "namespace": "format", + "defaultValue": "json" + }, + "appendProperties": { + "type": "mapProps", + "namespace": "appendproperty" + } + } + }, + "cosmosdb": { + "type": "object", + "namespace": "cosmosdb", + "fields": { + "connectionStringRef": "connectionstring", + "database": "database", + "collection": "collection" + } + }, + "httpPost": { + "type": "object", + "namespace": "httppost", + "fields": { + "endpoint": "endpoint", + "filter": "filter", + "appendHeaders": { + "type": "mapProps", + "namespace": "header" + } + } + } + } + } + } + } +} \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/customFlow.json b/Services/DataX.Config/DataX.Config.Test/Resource/customFlow.json new file mode 100644 index 000000000..d965da54c --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/Resource/customFlow.json @@ -0,0 +1,105 @@ +{ + "name": "customconfiggentest", + "icon": "/img/vs.png", + "displayName": "customconfiggentest", + "properties": { + "foo": "bar" + }, + "commonProcessor": { + "sparkJobConfigFolder": "wasbs://containername@someblob.core.windows.net/customconfiggentest", + "template": { + "name": "${jobName}", + "resources": "keyvault://somekeyvault/customconfiggentest-resources.xml", + "input": { + "sources": { + "myinput${sourceShardNumber}": { + "target": "mytarget${partitionJobNumber}" + } + }, + "eventhub": { + "consumerGroup": "customconfiggentest${partitionJobNumber}-staging", + "checkpointDir": "hdfs://mycluster/checkpoints/${jobName}", + "checkpointInterval": "300", + "connectionString": "keyvault://somekeyvault/customconfiggentest-eventhubconnectionstring", + "maxRate": "300", + "partitionCount": "10", + "startEnqueueTime": "-60" + }, + "streaming": { + "checkpointDir": "hdfs://mycluster/checkpoints/${jobName}", + "intervalInSeconds": 300 + }, + "blobSchemaFile": "keyvault://somekeyvault/customconfiggentest-rawSchema.json", + "eventNamePath": "eventhubpath", + "blobPathRegex": "\\d{4}/\\d{2}", + "fileTimeRegex": "\\d{4}/\\d{2}", + "sourceIdRegex": "\\d{4}/\\d{2}", + "fileTimeFormat": "yyyyMMdd", + "referenceData": [] + }, + "process": { + "metric": { + "eventhub": "keyvault://somekeyvault/metric-eventhubconnectionstring" + }, + "transform": "keyvault://somekeyvault/customconfiggentest-transform.xml", + "projections": [ + "keyvault://somekeyvault/customconfiggentest-projections.xml" + ] + }, + "output": { + "blob": { + "groups": { + "main": { + "folder": "keyvault://somekeyvault/customconfiggentest-bloboutput" + }, + "quarantined": { + "folder": "keyvault://somekeyvault/customconfiggentest-quarantined" + } + }, + "groupEvaluation": "test", + "format": "json", + "compressionType": "gzip" + }, + "name": "default", + "processedSchemaPath": "keyvault://somekeyvault/customconfiggentest-processedschemapath" + } + }, + "sparkJobTemplateRef": "defaultJobTemplate", + "jobCommonTokens": { + "jobName": "customconfiggentest${partitionJobNumber}", + "sparkJobName": "customconfiggentest${partitionJobNumber}", + "sparkJobConfigFileName": "customconfiggentest${partitionJobNumber}.json", + "sparkJobDriverLogLevel": "WARN", + "sparkJobExecutorLogLevel": "INFO", + "sparkJobNumExecutors": 6, + "sparkJobExecutorMemory": "8000m", + "processedSchemaPath": null + }, + "jobs": [ + { + "sourceShardNumber": "01", + "partitionJobNumber": "1", + "processedSchemaPath": null + }, + { + "sourceShardNumber": "02", + "partitionJobNumber": "2", + "processedSchemaPath": null + } + ] + }, + "metrics": { + "initParameters": { + "widgetSets": [ + "direct" + ], + "jobNames": { + "type": "getCPSparkJobNames" + } + } + }, + "jobNames": [ + "customconfiggentest1", + "customconfiggentest2" + ] +} \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/customJobConfig1.conf b/Services/DataX.Config/DataX.Config.Test/Resource/customJobConfig1.conf new file mode 100644 index 000000000..86fa1e08c --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/Resource/customJobConfig1.conf @@ -0,0 +1,33 @@ +#name of the datax job +datax.job.name=customconfiggentest1 + +#input configuration +datax.job.input.default.source.myinput01.target=mytarget1 +datax.job.input.default.eventhub.connectionstring=keyvault://somekeyvault/customconfiggentest-eventhubconnectionstring +datax.job.input.default.eventhub.consumergroup=customconfiggentest1-staging +datax.job.input.default.eventhub.checkpointdir=hdfs://mycluster/checkpoints/customconfiggentest1 +datax.job.input.default.eventhub.checkpointinterval=300 +datax.job.input.default.eventhub.maxrate=300 +datax.job.input.default.eventhub.startenqueuetime=-60 +datax.job.input.default.eventhub.partitioncount=10 +datax.job.input.default.streaming.checkpointdir=hdfs://mycluster/checkpoints/customconfiggentest1 +datax.job.input.default.streaming.intervalinseconds=300 +datax.job.input.default.sourceidregex=\d{4}/\d{2} +datax.job.input.default.blobschemafile=keyvault://somekeyvault/customconfiggentest-rawSchema.json +datax.job.input.default.eventnamepath=eventhubpath +datax.job.input.default.blobpathregex=\d{4}/\d{2} +datax.job.input.default.filetimeregex=\d{4}/\d{2} +datax.job.input.default.filetimeformat=yyyyMMdd + +## metric sender +datax.job.process.metric.eventhub=keyvault://somekeyvault/metric-eventhubconnectionstring + +## transform and projection +datax.job.process.projection=keyvault://somekeyvault/customconfiggentest-projections.xml +datax.job.process.transform=keyvault://somekeyvault/customconfiggentest-transform.xml + +# output settings +datax.job.output.default.processedschemapath=keyvault://somekeyvault/customconfiggentest-processedschemapath +datax.job.output.default.blob.groupevaluation=test +datax.job.output.default.blob.group.main.folder=keyvault://somekeyvault/customconfiggentest-bloboutput +datax.job.output.default.blob.group.quarantined.folder=keyvault://somekeyvault/customconfiggentest-quarantined \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/customJobConfig2.conf b/Services/DataX.Config/DataX.Config.Test/Resource/customJobConfig2.conf new file mode 100644 index 000000000..3dcfc190b --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/Resource/customJobConfig2.conf @@ -0,0 +1,33 @@ +#name of the datax job +datax.job.name=customconfiggentest2 + +#input configuration +datax.job.input.default.source.myinput02.target=mytarget2 +datax.job.input.default.eventhub.connectionstring=keyvault://somekeyvault/customconfiggentest-eventhubconnectionstring +datax.job.input.default.eventhub.consumergroup=customconfiggentest2-staging +datax.job.input.default.eventhub.checkpointdir=hdfs://mycluster/checkpoints/customconfiggentest2 +datax.job.input.default.eventhub.checkpointinterval=300 +datax.job.input.default.eventhub.maxrate=300 +datax.job.input.default.eventhub.startenqueuetime=-60 +datax.job.input.default.eventhub.partitioncount=10 +datax.job.input.default.streaming.checkpointdir=hdfs://mycluster/checkpoints/customconfiggentest2 +datax.job.input.default.streaming.intervalinseconds=300 +datax.job.input.default.sourceidregex=\d{4}/\d{2} +datax.job.input.default.blobschemafile=keyvault://somekeyvault/customconfiggentest-rawSchema.json +datax.job.input.default.eventnamepath=eventhubpath +datax.job.input.default.blobpathregex=\d{4}/\d{2} +datax.job.input.default.filetimeregex=\d{4}/\d{2} +datax.job.input.default.filetimeformat=yyyyMMdd + +## metric sender +datax.job.process.metric.eventhub=keyvault://somekeyvault/metric-eventhubconnectionstring + +## transform and projection +datax.job.process.projection=keyvault://somekeyvault/customconfiggentest-projections.xml +datax.job.process.transform=keyvault://somekeyvault/customconfiggentest-transform.xml + +# output settings +datax.job.output.default.processedschemapath=keyvault://somekeyvault/customconfiggentest-processedschemapath +datax.job.output.default.blob.groupevaluation=test +datax.job.output.default.blob.group.main.folder=keyvault://somekeyvault/customconfiggentest-bloboutput +datax.job.output.default.blob.group.quarantined.folder=keyvault://somekeyvault/customconfiggentest-quarantined \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/databricksFlowSaved.json b/Services/DataX.Config/DataX.Config.Test/Resource/databricksFlowSaved.json new file mode 100644 index 000000000..b5de0264b --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/Resource/databricksFlowSaved.json @@ -0,0 +1,406 @@ +{ + "name": "dbconfiggentest", + "icon": "/img/iot.png", + "displayName": "dbconfiggentest", + "properties": { + "foo": "bar" + }, + "gui": { + "name": "dbconfiggentest", + "displayName": "dbconfiggentest", + "databricksToken": "sometoken", + "owner": "data-accelerator@microsoft.com", + "input": { + "mode": "streaming", + "type": "iothub", + "properties": { + "inputEventhubName": "dbconfiggentest-iothub", + "inputEventhubConnection": "secretscope://somekeyvault/dbconfiggentest-input-eventhubconnectionstring", + "windowDuration": "60", + "timestampColumn": "eventTime", + "watermarkValue": "60", + "watermarkUnit": "second", + "maxRate": "35000", + "inputSchemaFile": "{\"type\":\"struct\",\"fields\":[{\"name\":\"deviceDetails\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"deviceId\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"deviceType\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"eventTime\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"homeId\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"status\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}", + "showNormalizationSnippet": false, + "normalizationSnippet": "stringToTimestamp(Raw.deviceDetails.eventTime) AS eventTimeStamp\r\nRaw.*", + "inputSubscriptionId": "acfaaa5f-a581-4375-badb-9ea61ffff0fc", + "inputResourceGroup": "TestRG" + }, + "referenceData": [ + { + "id": "testrd", + "type": "csv", + "properties": { + "path": "secretscope://somekeyvault/dbconfiggentest-referencedata-21083E2E44F15D315158D78A613254AA", + "delimiter": ",", + "header": true + }, + "typeDisplay": "CSV/TSV File" + } + ] + }, + "process": { + "timestampColumn": "eventTime", + "watermark": "60 second", + "functions": [ + { + "id": "myudaf", + "type": "jarUDAF", + "properties": { + "path": "secretscope://somekeyvault/dbconfiggentest-jarpath-2BE6B2F2C0AA1682F4E6EBF73B124D6E", + "class": "datax.udf.myudaf", + "libs": [], + "name": "myudaf" + }, + "typeDisplay": "UDAF" + }, + { + "id": "myudf", + "type": "jarUDF", + "properties": { + "path": "secretscope://somekeyvault/dbconfiggentest-jarpath-2BE6B2F2C0AA1682F4E6EBF73B124D6E", + "class": "datax.udf.myudf", + "libs": [], + "name": "myudf" + }, + "typeDisplay": "UDF" + }, + { + "id": "testaz", + "type": "azureFunction", + "properties": { + "serviceEndpoint": "https://asdfasdf.asdfasdf.as", + "api": "sdfsdf", + "code": "secretscope://somekeyvault/dbconfiggentest-azurefunc-4CDC36BB32FDD98D392C9C0F0675F44D", + "methodType": "get", + "params": [ + "params1" + ], + "name": "testaz" + }, + "typeDisplay": "Azure Function" + } + ], + "queries": [ + "--DataXStates--\nCREATE TABLE dbconfiggentest_garagedoor_accumulated\n (deviceId long, deviceType string, homeId long, MaxEventTime Timestamp, MinReading long, MaxReading long);\n\n--DataXQuery--\nNPI_DeviceWindow = SELECT \n deviceDetails.deviceId,\n deviceDetails.deviceType,\n eventTimeStamp,\n deviceDetails.homeId,\n deviceDetails.status\n FROM DataXProcessedInput \n TIMEWINDOW('5 minutes');\n\n--DataXQuery--\nNPI_DeviceWindow2 = SELECT \n deviceDetails.deviceId,\n deviceDetails.deviceType,\n eventTimeStamp,\n deviceDetails.homeId,\n deviceDetails.status\n FROM DataXProcessedInput \n TIMEWINDOW('2 minutes');\n\n--DataXQuery--\nDeviceInfoTimeWindow = SELECT \n deviceId,\n deviceType,\n homeId,\n MAX(eventTimeStamp) AS MaxEventTime,\n MIN(status) AS MinReading,\n MAX(status) AS MaxReading\n FROM NPI_DeviceWindow\n GROUP BY deviceId, deviceType, homeId;\n\n--DataXQuery--\nHeaterStatus = SELECT \n eventTimeStamp AS EventTime,\n 'HeaterStatus' AS MetricName,\n deviceDetails.status AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'Heating';\n \n--DataXQuery--\nWindowLockStatus = SELECT \n eventTimeStamp AS EventTime,\n 'WindowLockStatus' AS MetricName,\n deviceDetails.status AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'WindowLock';\n\n--DataXQuery--\nWindowLockSliding = SELECT\n MaxEventTime AS EventTime,\n 'WindowLockSliding' AS MetricName,\n MaxReading AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DeviceInfoTimeWindow\n INNER JOIN WindowLockStatus ON WindowLockStatus.EventTime = DeviceInfoTimeWindow.MaxEventTime\n WHERE homeId = 150\n\t\t\t\t\t\tAND deviceType = 'WindowLock';\n\n--DataXQuery--\nWindowOpenWhileHeaterOnAlert = SELECT\n MaxEventTime AS EventTime,\n 'WindowOpenWhileHeaterOnAlert' AS MetricName,\n 0 AS Metric,\n 'iotdevicesample' AS Product, \n 'Window open for 5+ minutes while heater is on.' AS Pivot1\n FROM DeviceInfoTimeWindow\n INNER JOIN WindowLockStatus ON WindowLockStatus.EventTime = DeviceInfoTimeWindow.MaxEventTime\n WHERE homeId = 150\n AND MaxReading = 0\n\t\t\t\t\t\t\t\t\tAND deviceType = 'WindowLock';\n\n--DataXQuery--\nDoorLockStatus = SELECT\n eventTimeStamp AS EventTime,\n 'DoorLockStatus' AS MetricName,\n deviceDetails.status AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'DoorLock';\n\n--DataXQuery--\nDoorLockOpenBetween9pmAnd7amUtcAlert = SELECT\n eventTimeStamp AS EventTime,\n 'DoorLockOpenBetween9pmAnd7amUtcAlert' AS MetricName,\n 0 AS Metric,\n 'iotdevicesample' AS Product, \n 'Door lock open between 9PM and 7AM UTC' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'DoorLock'\n AND deviceDetails.status = 0\n AND (hour(eventTimeStamp) >= 21 OR hour(eventTimeStamp) < 7);\n\n--DataXQuery--\nGarageDoorStatus = SELECT\n eventTimeStamp AS EventTime,\n 'GarageDoorStatus' AS MetricName,\n deviceDetails.status AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'GarageDoorLock';\n\n--DataXQuery--\nGarageDoorAccumalator = SELECT \n deviceId,\n deviceType,\n homeId,\n MaxEventTime,\n MinReading,\n MaxReading\n FROM DeviceInfoTimeWindow\n WHERE homeId = 150\n AND deviceType = 'GarageDoorLock'\n UNION ALL\n SELECT \n deviceId,\n deviceType,\n homeId,\n MaxEventTime,\n MinReading,\n MaxReading\n FROM iotdevicesample_GarageDoor_accumulated\n WHERE hour(MaxEventTime) = hour(current_timestamp());\n\n--DataXQuery--\nSELECT deviceId, deviceType, homeId, MaxEventTime, MinReading, MaxReading\nFROM GarageDoorAccumalator\nWITH UPSERT iotdevicesample_GarageDoor_accumulated;\n\n--DataXQuery--\nGarageDoorSliding = SELECT\n MaxEventTime AS EventTime,\n 'GarageDoorSliding' AS MetricName,\n MaxReading AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM iotdevicesample_GarageDoor_accumulated\n INNER JOIN GarageDoorStatus ON GarageDoorStatus.EventTime = iotdevicesample_GarageDoor_accumulated.MaxEventTime\n WHERE homeId = 150\n AND deviceType = 'GarageDoorLock';\n\n--DataXQuery--\nGarageDoorSlidingOpenInAnHour = SELECT COUNT(MaxReading) AS NumOfTimesGarageOpenFor5Mins,\n MAX(MaxEventTime) AS MaxEventTime\n FROM iotdevicesample_GarageDoor_accumulated\n WHERE homeId = 150\n AND deviceType = 'GarageDoorLock'\n AND MaxReading = 0;\n\n--DataXQuery--\nGarageOpenFor5MinsInAnHourAlert = SELECT \n MaxEventTime AS EventTime,\n 'GarageOpenFor5MinsInAnHourAlert' AS MetricName,\n 0 AS Metric,\n 'iotdevicesample' AS Product, \n CONCAT('No. of times garage door open for past 5 mins in current hour: ', NumOfTimesGarageOpenFor5Mins) AS Pivot1\n FROM GarageDoorSlidingOpenInAnHour;\n\nSimpleRulesTable = ProcessRules();\nAggregateRulesTable = ProcessAggregateRules(DataXProcessedInput);\nOUTPUT SimpleRulesTable TO myCosmosDB,myBlob;\nOUTPUT AggregateRulesTable TO myEventhub;\n\nOUTPUT HeaterStatus, WindowLockStatus, WindowLockSliding TO Metrics;\nOUTPUT WindowOpenWhileHeaterOnAlert TO Metrics;\nOUTPUT DoorLockStatus TO Metrics;\nOUTPUT DoorLockOpenBetween9pmAnd7amUtcAlert TO Metrics;\nOUTPUT GarageDoorStatus, GarageDoorSliding TO Metrics;\nOUTPUT GarageOpenFor5MinsInAnHourAlert TO Metrics;\n" + ], + "jobconfig": { + "jobNumExecutors": "6", + "jobExecutorMemory": "8000", + "jobDatabricksAutoScale": true, + "jobDatabricksMinWorkers": 1, + "jobDatabricksMaxWorkers": 8 + } + }, + "outputs": [ + { + "id": "Metrics", + "type": "metric", + "properties": {}, + "typeDisplay": "Metrics" + }, + { + "id": "myBlob", + "type": "blob", + "properties": { + "connectionString": "secretscope://somekeyvault/dbconfiggentest-output-5D09E8DD98332F8B2723EC5C1BCE9AD1", + "containerName": "outputs", + "blobPrefix": "testdifferentsub", + "blobPartitionFormat": "yyyy/MM/dd/HH", + "format": "json", + "compressionType": "none" + }, + "typeDisplay": "Azure Blob" + }, + { + "id": "myCosmosDB", + "type": "cosmosdb", + "properties": { + "connectionString": "secretscope://somekeyvault/dbconfiggentest-output-161E93325EAA303AD3816687341330A6", + "db": "Outputs", + "collection": "tests" + }, + "typeDisplay": "Cosmos DB" + }, + { + "id": "myEventhub", + "type": "eventHub", + "properties": { + "connectionString": "secretscope://somekeyvault/dbconfiggentest-output-67CEEABAAE420FB8C7A8FD11CCC23A30", + "format": "json", + "compressionType": "gzip" + }, + "typeDisplay": "Event Hub" + } + ], + "outputTemplates": [], + "rules": [ + { + "id": "DoorLock Close", + "type": "tagAlert", + "properties": { + "$productId": "iotdevicesample", + "$ruleType": "SimpleRule", + "$ruleId": "187a7066-d048-44a5-b08d-59158ec3d230", + "$ruleDescription": "DoorLock Close", + "$condition": "deviceDetails.deviceType = 'DoorLock' AND deviceDetails.status = 1", + "$tagName": "Tag", + "$tag": "CLOSE", + "$aggs": [], + "$pivots": [], + "$isAlert": false, + "$severity": "Critical", + "$alertSinks": [], + "$outputTemplate": "", + "schemaTableName": "DataXProcessedInput", + "conditions": { + "type": "group", + "conjunction": "or", + "conditions": [ + { + "type": "condition", + "conjunction": "or", + "aggregate": "AVG", + "field": "deviceDetails.deviceType", + "operator": "stringEqual", + "value": "DoorLock" + }, + { + "type": "condition", + "conjunction": "and", + "aggregate": "AVG", + "field": "deviceDetails.status", + "operator": "equal", + "value": "1" + } + ] + } + } + }, + { + "id": "DoorLock Open", + "type": "tagAlert", + "properties": { + "$productId": "iotdevicesample", + "$ruleType": "SimpleRule", + "$ruleId": "84f7b1c6-e501-448b-99f4-1a5cffa4b19d", + "$ruleDescription": "DoorLock Open", + "$condition": "deviceDetails.deviceType = 'DoorLock' AND deviceDetails.status = 0", + "$tagName": "Tag", + "$tag": "OPEN", + "$aggs": [], + "$pivots": [], + "$isAlert": false, + "$severity": "Critical", + "$alertSinks": [], + "$outputTemplate": "", + "schemaTableName": "DataXProcessedInput", + "conditions": { + "type": "group", + "conjunction": "or", + "conditions": [ + { + "type": "condition", + "conjunction": "or", + "aggregate": "AVG", + "field": "deviceDetails.deviceType", + "operator": "stringEqual", + "value": "DoorLock" + }, + { + "type": "condition", + "conjunction": "and", + "aggregate": "AVG", + "field": "deviceDetails.status", + "operator": "equal", + "value": "0" + } + ] + } + } + }, + { + "id": "Aggr", + "type": "tag", + "properties": { + "$productId": "dbconfiggentest", + "$ruleType": "AggregateRule", + "$ruleId": "5c85ede7-7a9a-4f64-992d-4c42dfd386c9", + "$ruleDescription": "Aggr", + "$condition": "AVG(deviceDetails.deviceId) > 2", + "$tagName": "Tag", + "$tag": "Tag2", + "$aggs": [ + "AVG(deviceDetails.deviceId)" + ], + "$pivots": [ "deviceDetails.deviceType" ], + "$isAlert": false, + "$severity": "Medium", + "$alertSinks": [ + "myBlob" + ], + "$outputTemplate": "", + "schemaTableName": "DataXProcessedInput", + "conditions": { + "type": "group", + "conjunction": "or", + "conditions": [ + { + "type": "condition", + "conjunction": "or", + "aggregate": "AVG", + "field": "deviceDetails.deviceId", + "operator": "greater", + "value": "2" + } + ] + } + } + }, + { + "id": "Test", + "type": "tag", + "properties": { + "$productId": "dbconfiggentest", + "$ruleType": "SimpleRule", + "$ruleId": "12574ff5-f41d-4169-8532-e9b450ac476f", + "$ruleDescription": "Test", + "$condition": "deviceDetails.deviceId = 9", + "$tagName": "Tag", + "$tag": "Tag1", + "$aggs": [], + "$pivots": [], + "$isAlert": false, + "$severity": "Critical", + "$alertSinks": [], + "$outputTemplate": "", + "schemaTableName": "DataXProcessedInput", + "conditions": { + "type": "group", + "conjunction": "or", + "conditions": [ + { + "type": "condition", + "conjunction": "or", + "aggregate": "AVG", + "field": "deviceDetails.deviceId", + "operator": "equal", + "value": "9" + } + ] + } + } + } + ], + "subscription": null + }, + "commonProcessor": { + "sparkJobConfigFolder": "file:///datax/runtime/testjobs/", + "template": { + "name": "${name}", + "input": { + "eventhub": { + "connectionString": "${inputEventHubConnectionString}", + "consumerGroup": "${inputEventHubConsumerGroup}", + "checkpointDir": "${inputEventHubCheckpointDir}", + "checkpointInterval": "${inputEventHubCheckpointInterval}", + "maxRate": "${inputEventHubMaxRate}", + "flushExistingCheckpoints": "${inputEventHubFlushExistingCheckpoints}" + }, + "streaming": { + "checkpointDir": "${inputStreamingCheckpointDir}", + "intervalInSeconds": "${inputStreamingIntervalInSeconds}" + }, + "blobSchemaFile": "${inputSchemaFilePath}", + "referenceData": "${inputReferenceData}" + }, + "process": { + "metric": { + "eventhub": "secretscope://somekeyvault/metric-eventhubconnectionstring" + }, + "timestampColumn": "${processTimestampColumn}", + "watermark": "${processWatermark}", + "jarUDAFs": "${processJarUDAFs}", + "jarUDFs": "${processJarUDFs}", + "azureFunctions": "${processAzureFunctions}", + "projections": "${processProjections}", + "timeWindows": "${processTimeWindows}", + "transform": "${processTransforms}", + "appendEventTags": {}, + "accumulationTables": "${processStateTables}" + }, + "outputs": "${outputs}" + }, + "sparkJobTemplateRef": "defaultJobTemplate", + "jobCommonTokens": { + "jobName": "${name}", + "sparkJobName": "DataXDirect-${name}", + "sparkJobDriverLogLevel": "WARN", + "sparkJobExecutorLogLevel": "INFO", + "sparkJobNumExecutors": "${guiSparkJobNumExecutors}", + "sparkJobExecutorMemory": "${guiSparkJobExecutorMemory}", + "processedSchemaPath": null + }, + "jobs": [ + { + "partitionJobNumber": "1" + } + ] + }, + "metrics": { + "sources": [ + { + "name": "events", + "input": { + "type": "MetricApi", + "metricKeys": [ + "DATAX-${name}:Input_DataXProcessedInput_Events_Count" + ] + }, + "output": { + "type": "SumWithTimeChart", + "data": { + "sum": true, + "timechart": true, + "average": true, + "speed": true + } + } + } + ], + "widgets": [ + { + "name": "eventsChart", + "displayName": "Events/Second", + "data": "events_timechart", + "position": "TimeCharts", + "type": "StackAreaChart" + }, + { + "name": "totalEvents", + "displayName": "Events Ingested Today", + "data": "events_sum", + "formatter": "longint", + "type": "SimpleBox", + "position": "FirstRow" + }, + { + "name": "averageEvents", + "displayName": "Avg. Events/Minute", + "data": "events_average", + "formatter": "longint", + "type": "SimpleBox", + "position": "FirstRow" + } + ], + "initParameters": { + "widgetSets": [ + "direct" + ], + "jobNames": { + "type": "getCPSparkJobNames" + } + } + } +} \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/databricksFlowStarted.json b/Services/DataX.Config/DataX.Config.Test/Resource/databricksFlowStarted.json new file mode 100644 index 000000000..c6de44bba --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/Resource/databricksFlowStarted.json @@ -0,0 +1,593 @@ +{ + "name": "dbconfiggentest", + "icon": "/img/iot.png", + "displayName": "dbconfiggentest", + "properties": { + "foo": "bar" + }, + "gui": { + "name": "dbconfiggentest", + "displayName": "dbconfiggentest", + "databricksToken": "sometoken", + "owner": "data-accelerator@microsoft.com", + "input": { + "mode": "streaming", + "type": "iothub", + "properties": { + "inputEventhubName": "dbconfiggentest-iothub", + "inputEventhubConnection": "secretscope://somekeyvault/dbconfiggentest-input-eventhubconnectionstring", + "windowDuration": "60", + "timestampColumn": "eventTime", + "watermarkValue": "60", + "watermarkUnit": "second", + "maxRate": "35000", + "inputSchemaFile": "{\"type\":\"struct\",\"fields\":[{\"name\":\"deviceDetails\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"deviceId\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"deviceType\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"eventTime\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"homeId\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"status\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}", + "showNormalizationSnippet": false, + "normalizationSnippet": "stringToTimestamp(Raw.deviceDetails.eventTime) AS eventTimeStamp\r\nRaw.*", + "inputSubscriptionId": "acfaaa5f-a581-4375-badb-9ea61ffff0fc", + "inputResourceGroup": "TestRG" + }, + "referenceData": [ + { + "id": "testrd", + "type": "csv", + "properties": { + "path": "secretscope://somekeyvault/dbconfiggentest-referencedata-21083E2E44F15D315158D78A613254AA", + "delimiter": ",", + "header": true + }, + "typeDisplay": "CSV/TSV File" + } + ] + }, + "process": { + "timestampColumn": "eventTime", + "watermark": "60 second", + "functions": [ + { + "id": "myudaf", + "type": "jarUDAF", + "properties": { + "path": "secretscope://somekeyvault/dbconfiggentest-jarpath-2BE6B2F2C0AA1682F4E6EBF73B124D6E", + "class": "datax.udf.myudaf", + "libs": [], + "name": "myudaf" + }, + "typeDisplay": "UDAF" + }, + { + "id": "myudf", + "type": "jarUDF", + "properties": { + "path": "secretscope://somekeyvault/dbconfiggentest-jarpath-2BE6B2F2C0AA1682F4E6EBF73B124D6E", + "class": "datax.udf.myudf", + "libs": [], + "name": "myudf" + }, + "typeDisplay": "UDF" + }, + { + "id": "testaz", + "type": "azureFunction", + "properties": { + "serviceEndpoint": "https://asdfasdf.asdfasdf.as", + "api": "sdfsdf", + "code": "secretscope://somekeyvault/dbconfiggentest-azurefunc-4CDC36BB32FDD98D392C9C0F0675F44D", + "methodType": "get", + "params": [ + "params1" + ], + "name": "testaz" + }, + "typeDisplay": "Azure Function" + } + ], + "queries": [ + "--DataXStates--\nCREATE TABLE dbconfiggentest_garagedoor_accumulated\n (deviceId long, deviceType string, homeId long, MaxEventTime Timestamp, MinReading long, MaxReading long);\n\n--DataXQuery--\nNPI_DeviceWindow = SELECT \n deviceDetails.deviceId,\n deviceDetails.deviceType,\n eventTimeStamp,\n deviceDetails.homeId,\n deviceDetails.status\n FROM DataXProcessedInput \n TIMEWINDOW('5 minutes');\n\n--DataXQuery--\nNPI_DeviceWindow2 = SELECT \n deviceDetails.deviceId,\n deviceDetails.deviceType,\n eventTimeStamp,\n deviceDetails.homeId,\n deviceDetails.status\n FROM DataXProcessedInput \n TIMEWINDOW('2 minutes');\n\n--DataXQuery--\nDeviceInfoTimeWindow = SELECT \n deviceId,\n deviceType,\n homeId,\n MAX(eventTimeStamp) AS MaxEventTime,\n MIN(status) AS MinReading,\n MAX(status) AS MaxReading\n FROM NPI_DeviceWindow\n GROUP BY deviceId, deviceType, homeId;\n\n--DataXQuery--\nHeaterStatus = SELECT \n eventTimeStamp AS EventTime,\n 'HeaterStatus' AS MetricName,\n deviceDetails.status AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'Heating';\n \n--DataXQuery--\nWindowLockStatus = SELECT \n eventTimeStamp AS EventTime,\n 'WindowLockStatus' AS MetricName,\n deviceDetails.status AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'WindowLock';\n\n--DataXQuery--\nWindowLockSliding = SELECT\n MaxEventTime AS EventTime,\n 'WindowLockSliding' AS MetricName,\n MaxReading AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DeviceInfoTimeWindow\n INNER JOIN WindowLockStatus ON WindowLockStatus.EventTime = DeviceInfoTimeWindow.MaxEventTime\n WHERE homeId = 150\n\t\t\t\t\t\tAND deviceType = 'WindowLock';\n\n--DataXQuery--\nWindowOpenWhileHeaterOnAlert = SELECT\n MaxEventTime AS EventTime,\n 'WindowOpenWhileHeaterOnAlert' AS MetricName,\n 0 AS Metric,\n 'iotdevicesample' AS Product, \n 'Window open for 5+ minutes while heater is on.' AS Pivot1\n FROM DeviceInfoTimeWindow\n INNER JOIN WindowLockStatus ON WindowLockStatus.EventTime = DeviceInfoTimeWindow.MaxEventTime\n WHERE homeId = 150\n AND MaxReading = 0\n\t\t\t\t\t\t\t\t\tAND deviceType = 'WindowLock';\n\n--DataXQuery--\nDoorLockStatus = SELECT\n eventTimeStamp AS EventTime,\n 'DoorLockStatus' AS MetricName,\n deviceDetails.status AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'DoorLock';\n\n--DataXQuery--\nDoorLockOpenBetween9pmAnd7amUtcAlert = SELECT\n eventTimeStamp AS EventTime,\n 'DoorLockOpenBetween9pmAnd7amUtcAlert' AS MetricName,\n 0 AS Metric,\n 'iotdevicesample' AS Product, \n 'Door lock open between 9PM and 7AM UTC' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'DoorLock'\n AND deviceDetails.status = 0\n AND (hour(eventTimeStamp) >= 21 OR hour(eventTimeStamp) < 7);\n\n--DataXQuery--\nGarageDoorStatus = SELECT\n eventTimeStamp AS EventTime,\n 'GarageDoorStatus' AS MetricName,\n deviceDetails.status AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM DataXProcessedInput\n WHERE deviceDetails.homeId = 150\n AND deviceDetails.deviceType = 'GarageDoorLock';\n\n--DataXQuery--\nGarageDoorAccumalator = SELECT \n deviceId,\n deviceType,\n homeId,\n MaxEventTime,\n MinReading,\n MaxReading\n FROM DeviceInfoTimeWindow\n WHERE homeId = 150\n AND deviceType = 'GarageDoorLock'\n UNION ALL\n SELECT \n deviceId,\n deviceType,\n homeId,\n MaxEventTime,\n MinReading,\n MaxReading\n FROM iotdevicesample_GarageDoor_accumulated\n WHERE hour(MaxEventTime) = hour(current_timestamp());\n\n--DataXQuery--\nSELECT deviceId, deviceType, homeId, MaxEventTime, MinReading, MaxReading\nFROM GarageDoorAccumalator\nWITH UPSERT iotdevicesample_GarageDoor_accumulated;\n\n--DataXQuery--\nGarageDoorSliding = SELECT\n MaxEventTime AS EventTime,\n 'GarageDoorSliding' AS MetricName,\n MaxReading AS Metric,\n 'iotdevicesample' AS Product, \n '' AS Pivot1\n FROM iotdevicesample_GarageDoor_accumulated\n INNER JOIN GarageDoorStatus ON GarageDoorStatus.EventTime = iotdevicesample_GarageDoor_accumulated.MaxEventTime\n WHERE homeId = 150\n AND deviceType = 'GarageDoorLock';\n\n--DataXQuery--\nGarageDoorSlidingOpenInAnHour = SELECT COUNT(MaxReading) AS NumOfTimesGarageOpenFor5Mins,\n MAX(MaxEventTime) AS MaxEventTime\n FROM iotdevicesample_GarageDoor_accumulated\n WHERE homeId = 150\n AND deviceType = 'GarageDoorLock'\n AND MaxReading = 0;\n\n--DataXQuery--\nGarageOpenFor5MinsInAnHourAlert = SELECT \n MaxEventTime AS EventTime,\n 'GarageOpenFor5MinsInAnHourAlert' AS MetricName,\n 0 AS Metric,\n 'iotdevicesample' AS Product, \n CONCAT('No. of times garage door open for past 5 mins in current hour: ', NumOfTimesGarageOpenFor5Mins) AS Pivot1\n FROM GarageDoorSlidingOpenInAnHour;\n\nSimpleRulesTable = ProcessRules();\nAggregateRulesTable = ProcessAggregateRules(DataXProcessedInput);\nOUTPUT SimpleRulesTable TO myCosmosDB,myBlob;\nOUTPUT AggregateRulesTable TO myEventhub;\n\nOUTPUT HeaterStatus, WindowLockStatus, WindowLockSliding TO Metrics;\nOUTPUT WindowOpenWhileHeaterOnAlert TO Metrics;\nOUTPUT DoorLockStatus TO Metrics;\nOUTPUT DoorLockOpenBetween9pmAnd7amUtcAlert TO Metrics;\nOUTPUT GarageDoorStatus, GarageDoorSliding TO Metrics;\nOUTPUT GarageOpenFor5MinsInAnHourAlert TO Metrics;\n" + ], + "jobconfig": { + "jobNumExecutors": "6", + "jobExecutorMemory": "8000", + "jobDatabricksAutoScale": true, + "jobDatabricksMinWorkers": 1, + "jobDatabricksMaxWorkers": 8 + } + }, + "outputs": [ + { + "id": "Metrics", + "type": "metric", + "properties": {}, + "typeDisplay": "Metrics" + }, + { + "id": "myBlob", + "type": "blob", + "properties": { + "connectionString": "secretscope://somekeyvault/dbconfiggentest-output-5D09E8DD98332F8B2723EC5C1BCE9AD1", + "containerName": "outputs", + "blobPrefix": "testdifferentsub", + "blobPartitionFormat": "yyyy/MM/dd/HH", + "format": "json", + "compressionType": "none" + }, + "typeDisplay": "Azure Blob" + }, + { + "id": "myCosmosDB", + "type": "cosmosdb", + "properties": { + "connectionString": "secretscope://somekeyvault/dbconfiggentest-output-161E93325EAA303AD3816687341330A6", + "db": "Outputs", + "collection": "tests" + }, + "typeDisplay": "Cosmos DB" + }, + { + "id": "myEventhub", + "type": "eventHub", + "properties": { + "connectionString": "secretscope://somekeyvault/dbconfiggentest-output-67CEEABAAE420FB8C7A8FD11CCC23A30", + "format": "json", + "compressionType": "gzip" + }, + "typeDisplay": "Event Hub" + } + ], + "outputTemplates": [], + "rules": [ + { + "id": "DoorLock Close", + "type": "tagAlert", + "properties": { + "$productId": "iotdevicesample", + "$ruleType": "SimpleRule", + "$ruleId": "187a7066-d048-44a5-b08d-59158ec3d230", + "$ruleDescription": "DoorLock Close", + "$condition": "deviceDetails.deviceType = 'DoorLock' AND deviceDetails.status = 1", + "$tagName": "Tag", + "$tag": "CLOSE", + "$aggs": [], + "$pivots": [], + "$isAlert": false, + "$severity": "Critical", + "$alertSinks": [], + "$outputTemplate": "", + "schemaTableName": "DataXProcessedInput", + "conditions": { + "type": "group", + "conjunction": "or", + "conditions": [ + { + "type": "condition", + "conjunction": "or", + "aggregate": "AVG", + "field": "deviceDetails.deviceType", + "operator": "stringEqual", + "value": "DoorLock" + }, + { + "type": "condition", + "conjunction": "and", + "aggregate": "AVG", + "field": "deviceDetails.status", + "operator": "equal", + "value": "1" + } + ] + } + } + }, + { + "id": "DoorLock Open", + "type": "tagAlert", + "properties": { + "$productId": "iotdevicesample", + "$ruleType": "SimpleRule", + "$ruleId": "84f7b1c6-e501-448b-99f4-1a5cffa4b19d", + "$ruleDescription": "DoorLock Open", + "$condition": "deviceDetails.deviceType = 'DoorLock' AND deviceDetails.status = 0", + "$tagName": "Tag", + "$tag": "OPEN", + "$aggs": [], + "$pivots": [], + "$isAlert": false, + "$severity": "Critical", + "$alertSinks": [], + "$outputTemplate": "", + "schemaTableName": "DataXProcessedInput", + "conditions": { + "type": "group", + "conjunction": "or", + "conditions": [ + { + "type": "condition", + "conjunction": "or", + "aggregate": "AVG", + "field": "deviceDetails.deviceType", + "operator": "stringEqual", + "value": "DoorLock" + }, + { + "type": "condition", + "conjunction": "and", + "aggregate": "AVG", + "field": "deviceDetails.status", + "operator": "equal", + "value": "0" + } + ] + } + } + }, + { + "id": "Aggr", + "type": "tag", + "properties": { + "$productId": "dbconfiggentest", + "$ruleType": "AggregateRule", + "$ruleId": "5c85ede7-7a9a-4f64-992d-4c42dfd386c9", + "$ruleDescription": "Aggr", + "$condition": "AVG(deviceDetails.deviceId) > 2", + "$tagName": "Tag", + "$tag": "Tag2", + "$aggs": [ + "AVG(deviceDetails.deviceId)" + ], + "$pivots": [ "deviceDetails.deviceType" ], + "$isAlert": false, + "$severity": "Medium", + "$alertSinks": [ + "myBlob" + ], + "$outputTemplate": "", + "schemaTableName": "DataXProcessedInput", + "conditions": { + "type": "group", + "conjunction": "or", + "conditions": [ + { + "type": "condition", + "conjunction": "or", + "aggregate": "AVG", + "field": "deviceDetails.deviceId", + "operator": "greater", + "value": "2" + } + ] + } + } + }, + { + "id": "Test", + "type": "tag", + "properties": { + "$productId": "dbconfiggentest", + "$ruleType": "SimpleRule", + "$ruleId": "12574ff5-f41d-4169-8532-e9b450ac476f", + "$ruleDescription": "Test", + "$condition": "deviceDetails.deviceId = 9", + "$tagName": "Tag", + "$tag": "Tag1", + "$aggs": [], + "$pivots": [], + "$isAlert": false, + "$severity": "Critical", + "$alertSinks": [], + "$outputTemplate": "", + "schemaTableName": "DataXProcessedInput", + "conditions": { + "type": "group", + "conjunction": "or", + "conditions": [ + { + "type": "condition", + "conjunction": "or", + "aggregate": "AVG", + "field": "deviceDetails.deviceId", + "operator": "equal", + "value": "9" + } + ] + } + } + } + ], + "subscription": null + }, + "commonProcessor": { + "sparkJobConfigFolder": "file:///datax/runtime/testjobs/", + "template": { + "name": "${name}", + "input": { + "eventhub": { + "connectionString": "${inputEventHubConnectionString}", + "consumerGroup": "${inputEventHubConsumerGroup}", + "checkpointDir": "${inputEventHubCheckpointDir}", + "checkpointInterval": "${inputEventHubCheckpointInterval}", + "maxRate": "${inputEventHubMaxRate}", + "flushExistingCheckpoints": "${inputEventHubFlushExistingCheckpoints}" + }, + "streaming": { + "checkpointDir": "${inputStreamingCheckpointDir}", + "intervalInSeconds": "${inputStreamingIntervalInSeconds}" + }, + "blobSchemaFile": "${inputSchemaFilePath}", + "referenceData": "${inputReferenceData}" + }, + "process": { + "metric": { + "eventhub": "secretscope://somekeyvault/metric-eventhubconnectionstring" + }, + "timestampColumn": "${processTimestampColumn}", + "watermark": "${processWatermark}", + "jarUDAFs": "${processJarUDAFs}", + "jarUDFs": "${processJarUDFs}", + "azureFunctions": "${processAzureFunctions}", + "projections": "${processProjections}", + "timeWindows": "${processTimeWindows}", + "transform": "${processTransforms}", + "appendEventTags": {}, + "accumulationTables": "${processStateTables}" + }, + "outputs": "${outputs}" + }, + "sparkJobTemplateRef": "defaultJobTemplate", + "jobCommonTokens": { + "jobName": "${name}", + "sparkJobName": "DataXDirect-${name}", + "sparkJobDriverLogLevel": "WARN", + "sparkJobExecutorLogLevel": "INFO", + "sparkJobNumExecutors": "${guiSparkJobNumExecutors}", + "sparkJobExecutorMemory": "${guiSparkJobExecutorMemory}", + "processedSchemaPath": null + }, + "jobs": [ + { + "partitionJobNumber": "1" + } + ] + }, + "metrics": { + "sources": [ + { + "name": "events", + "input": { + "type": "MetricApi", + "metricKeys": [ + "DATAX-dbconfiggentest:Input_DataXProcessedInput_Events_Count" + ] + }, + "output": { + "type": "SumWithTimeChart", + "data": { + "sum": true, + "timechart": true, + "average": true, + "speed": true + } + } + }, + { + "name": "HeaterStatus, WindowLockStatus, WindowLockSliding", + "input": { + "type": "MetricApi", + "pollingInterval": 60000, + "metricKeys": [ + { + "name": "dbconfiggentest:HeaterStatus", + "displayName": "HeaterStatus" + }, + { + "name": "dbconfiggentest:WindowLockStatus", + "displayName": "WindowLockStatus" + }, + { + "name": "dbconfiggentest:WindowLockSliding", + "displayName": "WindowLockSliding" + } + ] + }, + "output": { + "type": "DirectTimeChart", + "data": { + "timechart": true, + "current": false, + "table": false + }, + "chartTimeWindowInMs": 3600000 + } + }, + { + "name": "WindowOpenWhileHeaterOnAlert", + "input": { + "type": "MetricDetailsApi", + "pollingInterval": 60000, + "metricKeys": [ + { + "name": "dbconfiggentest:WindowOpenWhileHeaterOnAlert", + "displayName": "WindowOpenWhileHeaterOnAlert" + } + ] + }, + "output": { + "type": "DirectTable", + "data": { + "timechart": false, + "current": false, + "table": true + }, + "chartTimeWindowInMs": 3600000 + } + }, + { + "name": "DoorLockStatus", + "input": { + "type": "MetricApi", + "pollingInterval": 60000, + "metricKeys": [ + { + "name": "dbconfiggentest:DoorLockStatus", + "displayName": "DoorLockStatus" + } + ] + }, + "output": { + "type": "DirectTimeChart", + "data": { + "timechart": true, + "current": false, + "table": false + }, + "chartTimeWindowInMs": 3600000 + } + }, + { + "name": "DoorLockOpenBetween9pmAnd7amUtcAlert", + "input": { + "type": "MetricDetailsApi", + "pollingInterval": 60000, + "metricKeys": [ + { + "name": "dbconfiggentest:DoorLockOpenBetween9pmAnd7amUtcAlert", + "displayName": "DoorLockOpenBetween9pmAnd7amUtcAlert" + } + ] + }, + "output": { + "type": "DirectTable", + "data": { + "timechart": false, + "current": false, + "table": true + }, + "chartTimeWindowInMs": 3600000 + } + }, + { + "name": "GarageDoorStatus, GarageDoorSliding", + "input": { + "type": "MetricApi", + "pollingInterval": 60000, + "metricKeys": [ + { + "name": "dbconfiggentest:GarageDoorStatus", + "displayName": "GarageDoorStatus" + }, + { + "name": "dbconfiggentest:GarageDoorSliding", + "displayName": "GarageDoorSliding" + } + ] + }, + "output": { + "type": "DirectTimeChart", + "data": { + "timechart": true, + "current": false, + "table": false + }, + "chartTimeWindowInMs": 3600000 + } + }, + { + "name": "GarageOpenFor5MinsInAnHourAlert", + "input": { + "type": "MetricDetailsApi", + "pollingInterval": 60000, + "metricKeys": [ + { + "name": "dbconfiggentest:GarageOpenFor5MinsInAnHourAlert", + "displayName": "GarageOpenFor5MinsInAnHourAlert" + } + ] + }, + "output": { + "type": "DirectTable", + "data": { + "timechart": false, + "current": false, + "table": true + }, + "chartTimeWindowInMs": 3600000 + } + } + ], + "widgets": [ + { + "name": "eventsChart", + "displayName": "Events/Second", + "data": "events_timechart", + "position": "TimeCharts", + "type": "StackAreaChart" + }, + { + "name": "totalEvents", + "displayName": "Events Ingested Today", + "data": "events_sum", + "formatter": "longint", + "type": "SimpleBox", + "position": "FirstRow" + }, + { + "name": "averageEvents", + "displayName": "Avg. Events/Minute", + "data": "events_average", + "formatter": "longint", + "type": "SimpleBox", + "position": "FirstRow" + }, + { + "name": "HeaterStatus, WindowLockStatus, WindowLockSliding", + "data": "HeaterStatus, WindowLockStatus, WindowLockSliding_timechart", + "displayName": "HeaterStatus, WindowLockStatus, WindowLockSliding", + "position": "TimeCharts", + "type": "MultiLineChart" + }, + { + "name": "WindowOpenWhileHeaterOnAlert", + "data": "WindowOpenWhileHeaterOnAlert_table", + "displayName": "WindowOpenWhileHeaterOnAlert", + "position": "TimeCharts", + "type": "DetailsList" + }, + { + "name": "DoorLockStatus", + "data": "DoorLockStatus_timechart", + "displayName": "DoorLockStatus", + "position": "TimeCharts", + "type": "MultiLineChart" + }, + { + "name": "DoorLockOpenBetween9pmAnd7amUtcAlert", + "data": "DoorLockOpenBetween9pmAnd7amUtcAlert_table", + "displayName": "DoorLockOpenBetween9pmAnd7amUtcAlert", + "position": "TimeCharts", + "type": "DetailsList" + }, + { + "name": "GarageDoorStatus, GarageDoorSliding", + "data": "GarageDoorStatus, GarageDoorSliding_timechart", + "displayName": "GarageDoorStatus, GarageDoorSliding", + "position": "TimeCharts", + "type": "MultiLineChart" + }, + { + "name": "GarageOpenFor5MinsInAnHourAlert", + "data": "GarageOpenFor5MinsInAnHourAlert_table", + "displayName": "GarageOpenFor5MinsInAnHourAlert", + "position": "TimeCharts", + "type": "DetailsList" + } + ], + "initParameters": { + "widgetSets": [ + "direct" + ], + "jobNames": { + "type": "getCPSparkJobNames" + } + } + }, + "jobNames": [ "dbconfiggentest" ] +} \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/databricksJobConfig.conf b/Services/DataX.Config/DataX.Config.Test/Resource/databricksJobConfig.conf new file mode 100644 index 000000000..a0f76b7bc --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/Resource/databricksJobConfig.conf @@ -0,0 +1,75 @@ +# Configuration settings for the job +# 2019-07-24 - created +# ============================================================================ + +# Name of the job +# ---------------------------------------------------------------------------- +datax.job.name=dbconfiggentest + +# Input settings +# ---------------------------------------------------------------------------- +datax.job.input.default.eventhub.consumergroup=dbconfiggentest +datax.job.input.default.eventhub.checkpointdir=hdfs://mycluster/dataxdirect/dbconfiggentest/eventhub/checkpoints +datax.job.input.default.eventhub.checkpointinterval=60 +datax.job.input.default.eventhub.maxrate=35000 +datax.job.input.default.eventhub.connectionstring=secretscope://somekeyvault/dbconfiggentest-input-eventhubconnectionstring +datax.job.input.default.eventhub.flushexistingcheckpoints=True +datax.job.input.default.streaming.checkpointdir=dbfs:/mycluster/dataxdirect/dbconfiggentest/streaming/checkpoints +datax.job.input.default.streaming.intervalinseconds=60 +datax.job.input.default.blobschemafile=secretscope://somekeyvault/dbconfiggentest-inputschemafile +datax.job.input.default.referencedata.testrd.path=secretscope://somekeyvault/dbconfiggentest-referencedata-21083E2E44F15D315158D78A613254AA +datax.job.input.default.referencedata.testrd.format=csv +datax.job.input.default.referencedata.testrd.delimiter=, +datax.job.input.default.referencedata.testrd.header=True + +# Process settings +# ---------------------------------------------------------------------------- +datax.job.process.projection=secretscope://somekeyvault/dbconfiggentest-projectionfile +datax.job.process.transform=secretscope://somekeyvault/dbconfiggentest-transform +datax.job.process.timestampcolumn=eventTime +datax.job.process.watermark=60 second +#datax.job.process.timewindow.testvalue=10 seconds +#datax.job.process.statetable.testvalue.schema=.. +#datax.job.process.statetable.testvalue.location=.. +datax.job.process.timewindow.DataXProcessedInput_5minutes.windowduration=5 minutes +datax.job.process.timewindow.DataXProcessedInput_2minutes.windowduration=2 minutes +datax.job.process.jar.udaf.myudaf.class=datax.udf.myudaf +datax.job.process.jar.udaf.myudaf.path=secretscope://somekeyvault/dbconfiggentest-jarpath-2BE6B2F2C0AA1682F4E6EBF73B124D6E +datax.job.process.jar.udf.myudf.class=datax.udf.myudf +datax.job.process.jar.udf.myudf.path=secretscope://somekeyvault/dbconfiggentest-jarpath-2BE6B2F2C0AA1682F4E6EBF73B124D6E +datax.job.process.azurefunction.testaz.serviceendpoint=https://asdfasdf.asdfasdf.as +datax.job.process.azurefunction.testaz.api=sdfsdf +datax.job.process.azurefunction.testaz.code=secretscope://somekeyvault/dbconfiggentest-azurefunc-4CDC36BB32FDD98D392C9C0F0675F44D +datax.job.process.azurefunction.testaz.methodtype=get +datax.job.process.azurefunction.testaz.params=params1 +datax.job.process.statetable.dbconfiggentest_garagedoor_accumulated.schema=deviceId long, deviceType string, homeId long, MaxEventTime Timestamp, MinReading long, MaxReading long +datax.job.process.statetable.dbconfiggentest_garagedoor_accumulated.location=dbfs:/mycluster/datax/dbconfiggentest/dbconfiggentest_garagedoor_accumulated/ +# +datax.job.process.metric.eventhub=secretscope://somekeyvault/metric-eventhubconnectionstring + +# Output settings +# ---------------------------------------------------------------------------- +datax.job.output.SimpleRulesTable.blob.compressiontype=none +datax.job.output.SimpleRulesTable.blob.group.main.folder=secretscope://somekeyvault/dbconfiggentest-output-D7B7107DC4C5A693C746B31988180C1E +datax.job.output.SimpleRulesTable.cosmosdb.connectionstring=secretscope://somekeyvault/dbconfiggentest-output-161E93325EAA303AD3816687341330A6 +datax.job.output.SimpleRulesTable.cosmosdb.database=Outputs +datax.job.output.SimpleRulesTable.cosmosdb.collection=tests +datax.job.output.AggregateRulesTable.eventhub.connectionstring=secretscope://somekeyvault/dbconfiggentest-output-67CEEABAAE420FB8C7A8FD11CCC23A30 +datax.job.output.HeaterStatus.eventhub.connectionstring=secretscope://somekeyvault/metric-eventhubconnectionstring +datax.job.output.HeaterStatus.eventhub.compressiontype=none +datax.job.output.WindowLockStatus.eventhub.connectionstring=secretscope://somekeyvault/metric-eventhubconnectionstring +datax.job.output.WindowLockStatus.eventhub.compressiontype=none +datax.job.output.WindowLockSliding.eventhub.connectionstring=secretscope://somekeyvault/metric-eventhubconnectionstring +datax.job.output.WindowLockSliding.eventhub.compressiontype=none +datax.job.output.WindowOpenWhileHeaterOnAlert.eventhub.connectionstring=secretscope://somekeyvault/metric-eventhubconnectionstring +datax.job.output.WindowOpenWhileHeaterOnAlert.eventhub.compressiontype=none +datax.job.output.DoorLockStatus.eventhub.connectionstring=secretscope://somekeyvault/metric-eventhubconnectionstring +datax.job.output.DoorLockStatus.eventhub.compressiontype=none +datax.job.output.DoorLockOpenBetween9pmAnd7amUtcAlert.eventhub.connectionstring=secretscope://somekeyvault/metric-eventhubconnectionstring +datax.job.output.DoorLockOpenBetween9pmAnd7amUtcAlert.eventhub.compressiontype=none +datax.job.output.GarageDoorStatus.eventhub.connectionstring=secretscope://somekeyvault/metric-eventhubconnectionstring +datax.job.output.GarageDoorStatus.eventhub.compressiontype=none +datax.job.output.GarageDoorSliding.eventhub.connectionstring=secretscope://somekeyvault/metric-eventhubconnectionstring +datax.job.output.GarageDoorSliding.eventhub.compressiontype=none +datax.job.output.GarageOpenFor5MinsInAnHourAlert.eventhub.connectionstring=secretscope://somekeyvault/metric-eventhubconnectionstring +datax.job.output.GarageOpenFor5MinsInAnHourAlert.eventhub.compressiontype=none \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/databricksJobConfig.json b/Services/DataX.Config/DataX.Config.Test/Resource/databricksJobConfig.json new file mode 100644 index 000000000..478511ab8 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/Resource/databricksJobConfig.json @@ -0,0 +1,183 @@ +{ + "name": "dbconfiggentest", + "input": { + "eventhub": { + "connectionString": "secretscope://somekeyvault/dbconfiggentest-input-eventhubconnectionstring", + "consumerGroup": "dbconfiggentest", + "checkpointDir": "hdfs://mycluster/dataxdirect/dbconfiggentest/eventhub/checkpoints", + "checkpointInterval": "60", + "maxRate": "35000", + "flushExistingCheckpoints": true + }, + "streaming": { + "checkpointDir": "dbfs:/mycluster/dataxdirect/dbconfiggentest/streaming/checkpoints", + "intervalInSeconds": 60 + }, + "blobSchemaFile": "secretscope://somekeyvault/dbconfiggentest-inputschemafile", + "referenceData": [ + { + "format": "csv", + "name": "testrd", + "path": "secretscope://somekeyvault/dbconfiggentest-referencedata-21083E2E44F15D315158D78A613254AA", + "delimiter": ",", + "header": "True" + } + ] + }, + "process": { + "metric": { + "eventhub": "secretscope://somekeyvault/metric-eventhubconnectionstring" + }, + "timestampColumn": "eventTime", + "watermark": "60 second", + "jarUDAFs": [ + { + "path": "secretscope://somekeyvault/dbconfiggentest-jarpath-2BE6B2F2C0AA1682F4E6EBF73B124D6E", + "class": "datax.udf.myudaf", + "libs": [], + "name": "myudaf" + } + ], + "jarUDFs": [ + { + "path": "secretscope://somekeyvault/dbconfiggentest-jarpath-2BE6B2F2C0AA1682F4E6EBF73B124D6E", + "class": "datax.udf.myudf", + "libs": [], + "name": "myudf" + } + ], + "azureFunctions": [ + { + "serviceEndpoint": "https://asdfasdf.asdfasdf.as", + "api": "sdfsdf", + "code": "secretscope://somekeyvault/dbconfiggentest-azurefunc-4CDC36BB32FDD98D392C9C0F0675F44D", + "methodType": "get", + "params": [ + "params1" + ], + "name": "testaz" + } + ], + "projections": [ + "secretscope://somekeyvault/dbconfiggentest-projectionfile" + ], + "timeWindows": [ + { + "name": "DataXProcessedInput_5minutes", + "windowDuration": "5 minutes" + }, + { + "name": "DataXProcessedInput_2minutes", + "windowDuration": "2 minutes" + } + ], + "transform": "secretscope://somekeyvault/dbconfiggentest-transform", + "appendEventTags": {}, + "accumulationTables": [ + { + "name": "dbconfiggentest_garagedoor_accumulated", + "schema": "deviceId long, deviceType string, homeId long, MaxEventTime Timestamp, MinReading long, MaxReading long", + "location": "dbfs:/mycluster/datax/dbconfiggentest/dbconfiggentest_garagedoor_accumulated/" + } + ] + }, + "outputs": [ + { + "name": "SimpleRulesTable", + "blob": { + "groups": { + "main": { + "folder": "secretscope://somekeyvault/dbconfiggentest-output-D7B7107DC4C5A693C746B31988180C1E" + } + }, + "compressionType": "none", + "format": "json" + }, + "cosmosdb": { + "connectionStringRef": "secretscope://somekeyvault/dbconfiggentest-output-161E93325EAA303AD3816687341330A6", + "database": "Outputs", + "collection": "tests" + } + }, + { + "name": "AggregateRulesTable", + "eventhub": { + "connectionStringRef": "secretscope://somekeyvault/dbconfiggentest-output-67CEEABAAE420FB8C7A8FD11CCC23A30", + "compressionType": "gzip", + "format": "json" + } + }, + { + "name": "HeaterStatus", + "eventhub": { + "connectionStringRef": "secretscope://somekeyvault/metric-eventhubconnectionstring", + "compressionType": "none", + "format": "json" + } + }, + { + "name": "WindowLockStatus", + "eventhub": { + "connectionStringRef": "secretscope://somekeyvault/metric-eventhubconnectionstring", + "compressionType": "none", + "format": "json" + } + }, + { + "name": "WindowLockSliding", + "eventhub": { + "connectionStringRef": "secretscope://somekeyvault/metric-eventhubconnectionstring", + "compressionType": "none", + "format": "json" + } + }, + { + "name": "WindowOpenWhileHeaterOnAlert", + "eventhub": { + "connectionStringRef": "secretscope://somekeyvault/metric-eventhubconnectionstring", + "compressionType": "none", + "format": "json" + } + }, + { + "name": "DoorLockStatus", + "eventhub": { + "connectionStringRef": "secretscope://somekeyvault/metric-eventhubconnectionstring", + "compressionType": "none", + "format": "json" + } + }, + { + "name": "DoorLockOpenBetween9pmAnd7amUtcAlert", + "eventhub": { + "connectionStringRef": "secretscope://somekeyvault/metric-eventhubconnectionstring", + "compressionType": "none", + "format": "json" + } + }, + { + "name": "GarageDoorStatus", + "eventhub": { + "connectionStringRef": "secretscope://somekeyvault/metric-eventhubconnectionstring", + "compressionType": "none", + "format": "json" + } + }, + { + "name": "GarageDoorSliding", + "eventhub": { + "connectionStringRef": "secretscope://somekeyvault/metric-eventhubconnectionstring", + "compressionType": "none", + "format": "json" + } + }, + { + "name": "GarageOpenFor5MinsInAnHourAlert", + "eventhub": { + "connectionStringRef": "secretscope://somekeyvault/metric-eventhubconnectionstring", + "compressionType": "none", + "format": "json" + } + } + ] +} \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/dbconfiggentest-combined.txt b/Services/DataX.Config/DataX.Config.Test/Resource/dbconfiggentest-combined.txt new file mode 100644 index 000000000..298f18c5b --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/Resource/dbconfiggentest-combined.txt @@ -0,0 +1,335 @@ +--DataXQuery-- + NPI_DeviceWindow = SELECT + deviceDetails.deviceId, + deviceDetails.deviceType, + eventTimeStamp, + deviceDetails.homeId, + deviceDetails.status + FROM + DataXProcessedInput_5minutes + +--DataXQuery-- + NPI_DeviceWindow2 = SELECT + deviceDetails.deviceId, + deviceDetails.deviceType, + eventTimeStamp, + deviceDetails.homeId, + deviceDetails.status + FROM + DataXProcessedInput_2minutes + +--DataXQuery-- + DeviceInfoTimeWindow = SELECT + deviceId, + deviceType, + homeId, + MAX(eventTimeStamp) AS MaxEventTime, + MIN(status) AS MinReading, + MAX(status) AS MaxReading + FROM + NPI_DeviceWindow + GROUP BY + deviceId, + deviceType, + homeId + +--DataXQuery-- + HeaterStatus = SELECT + eventTimeStamp AS EventTime, + 'HeaterStatus' AS MetricName, + deviceDetails.status AS Metric, + 'iotdevicesample' AS Product, + '' AS Pivot1 + FROM + DataXProcessedInput + WHERE + deviceDetails.homeId = 150 + AND deviceDetails.deviceType = 'Heating' + +--DataXQuery-- + WindowLockStatus = SELECT + eventTimeStamp AS EventTime, + 'WindowLockStatus' AS MetricName, + deviceDetails.status AS Metric, + 'iotdevicesample' AS Product, + '' AS Pivot1 + FROM + DataXProcessedInput + WHERE + deviceDetails.homeId = 150 + AND deviceDetails.deviceType = 'WindowLock' + +--DataXQuery-- + WindowLockSliding = SELECT + MaxEventTime AS EventTime, + 'WindowLockSliding' AS MetricName, + MaxReading AS Metric, + 'iotdevicesample' AS Product, + '' AS Pivot1 + FROM + DeviceInfoTimeWindow + INNER JOIN + WindowLockStatus + ON WindowLockStatus.EventTime = DeviceInfoTimeWindow.MaxEventTime + WHERE + homeId = 150 + AND deviceType = 'WindowLock' + +--DataXQuery-- + WindowOpenWhileHeaterOnAlert = SELECT + MaxEventTime AS EventTime, + 'WindowOpenWhileHeaterOnAlert' AS MetricName, + 0 AS Metric, + 'iotdevicesample' AS Product, + 'Window open for 5+ minutes while heater is on.' AS Pivot1 + FROM + DeviceInfoTimeWindow + INNER JOIN + WindowLockStatus + ON WindowLockStatus.EventTime = DeviceInfoTimeWindow.MaxEventTime + WHERE + homeId = 150 + AND MaxReading = 0 + AND deviceType = 'WindowLock' + +--DataXQuery-- + DoorLockStatus = SELECT + eventTimeStamp AS EventTime, + 'DoorLockStatus' AS MetricName, + deviceDetails.status AS Metric, + 'iotdevicesample' AS Product, + '' AS Pivot1 + FROM + DataXProcessedInput + WHERE + deviceDetails.homeId = 150 + AND deviceDetails.deviceType = 'DoorLock' + +--DataXQuery-- + DoorLockOpenBetween9pmAnd7amUtcAlert = SELECT + eventTimeStamp AS EventTime, + 'DoorLockOpenBetween9pmAnd7amUtcAlert' AS MetricName, + 0 AS Metric, + 'iotdevicesample' AS Product, + 'Door lock open between 9PM and 7AM UTC' AS Pivot1 + FROM + DataXProcessedInput + WHERE + deviceDetails.homeId = 150 + AND deviceDetails.deviceType = 'DoorLock' + AND deviceDetails.status = 0 + AND ( + hour(eventTimeStamp) >= 21 + OR hour(eventTimeStamp) < 7 + ) + +--DataXQuery-- + GarageDoorStatus = SELECT + eventTimeStamp AS EventTime, + 'GarageDoorStatus' AS MetricName, + deviceDetails.status AS Metric, + 'iotdevicesample' AS Product, + '' AS Pivot1 + FROM + DataXProcessedInput + WHERE + deviceDetails.homeId = 150 + AND deviceDetails.deviceType = 'GarageDoorLock' + +--DataXQuery-- + GarageDoorAccumalator = SELECT + deviceId, + deviceType, + homeId, + MaxEventTime, + MinReading, + MaxReading + FROM + DeviceInfoTimeWindow + WHERE + homeId = 150 + AND deviceType = 'GarageDoorLock' + UNION + ALL SELECT + deviceId, + deviceType, + homeId, + MaxEventTime, + MinReading, + MaxReading + FROM + iotdevicesample_GarageDoor_accumulated + WHERE + hour(MaxEventTime) = hour(current_timestamp()) + +--DataXQuery-- + iotdevicesample_GarageDoor_accumulated = SELECT + deviceId, + deviceType, + homeId, + MaxEventTime, + MinReading, + MaxReading + FROM + GarageDoorAccumalator + +--DataXQuery-- + GarageDoorSliding = SELECT + MaxEventTime AS EventTime, + 'GarageDoorSliding' AS MetricName, + MaxReading AS Metric, + 'iotdevicesample' AS Product, + '' AS Pivot1 + FROM + iotdevicesample_GarageDoor_accumulated + INNER JOIN + GarageDoorStatus + ON GarageDoorStatus.EventTime = iotdevicesample_GarageDoor_accumulated.MaxEventTime + WHERE + homeId = 150 + AND deviceType = 'GarageDoorLock' + +--DataXQuery-- + GarageDoorSlidingOpenInAnHour = SELECT + COUNT(MaxReading) AS NumOfTimesGarageOpenFor5Mins, + MAX(MaxEventTime) AS MaxEventTime + FROM + iotdevicesample_GarageDoor_accumulated + WHERE + homeId = 150 + AND deviceType = 'GarageDoorLock' + AND MaxReading = 0 + +--DataXQuery-- + GarageOpenFor5MinsInAnHourAlert = SELECT + MaxEventTime AS EventTime, + 'GarageOpenFor5MinsInAnHourAlert' AS MetricName, + 0 AS Metric, + 'iotdevicesample' AS Product, + CONCAT('No. of times garage door open for past 5 mins in current hour: ', + NumOfTimesGarageOpenFor5Mins) AS Pivot1 + FROM + GarageDoorSlidingOpenInAnHour + +--DataXQuery-- + SimpleRulesTable = SELECT + *, + filterNull( + Array( + IF( + deviceDetails.deviceType = 'DoorLock' + AND deviceDetails.status = 1, + MAP( + 'ruleId', + '187a7066-d048-44a5-b08d-59158ec3d230', + 'ruleDescription', + 'DoorLock Close', + 'severity', + 'Critical', + 'Tag', + 'CLOSE' + ), + NULL + ), + IF( + deviceDetails.deviceType = 'DoorLock' + AND deviceDetails.status = 0, + MAP( + 'ruleId', + '84f7b1c6-e501-448b-99f4-1a5cffa4b19d', + 'ruleDescription', + 'DoorLock Open', + 'severity', + 'Critical', + 'Tag', + 'OPEN' + ), + NULL + ), + IF( + deviceDetails.deviceId = 9, + MAP( + 'ruleId', + '12574ff5-f41d-4169-8532-e9b450ac476f', + 'ruleDescription', + 'Test', + 'severity', + 'Critical', + 'Tag', + 'Tag1' + ), + NULL + ) + ) + ) AS Rules + FROM + DataXProcessedInput + + +--DataXQuery-- + ar1_2_1 = SELECT + AVG(deviceDetails.deviceId) AS deviceDetailsdeviceId_AVG, + deviceDetails.deviceType, + COUNT(*) AS Count + FROM + DataXProcessedInput + GROUP BY + deviceDetails.deviceType + +--DataXQuery-- + ar2_2_1 = SELECT + *, + IF( + deviceDetailsdeviceId_AVG > 2, + MAP( + 'ruleId', + '5c85ede7-7a9a-4f64-992d-4c42dfd386c9', + 'ruleDescription', + 'Aggr', + 'severity', + 'Medium', + 'Tag', + 'Tag2' + ), + NULL + ) AS RuleObject + FROM + ar1_2_1 + +--DataXQuery-- + ar3_2_1 = SELECT + MAP( + 'deviceDetails.deviceType', + deviceType + ) AS pivots, + MAP( + 'deviceDetails.deviceId', + MAP( + 'AVG', + deviceDetailsdeviceId_AVG + ) + ) AS aggs, + Count AS count, + MAP( + 'ruleId', + '5c85ede7-7a9a-4f64-992d-4c42dfd386c9', + 'Tag', + 'Tag2', + 'description', + 'Aggr', + 'severity', + 'Medium' + ) AS result + FROM + ar2_2_1 + +--DataXQuery-- + ar4_2 = SELECT + * + FROM + ar3_2_1 +--DataXQuery-- + AggregateRulesTable = SELECT + * + FROM + ar4_2 \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/flattenerConfig.json b/Services/DataX.Config/DataX.Config.Test/Resource/flattenerConfig.json index fde729184..745575049 100644 --- a/Services/DataX.Config/DataX.Config.Test/Resource/flattenerConfig.json +++ b/Services/DataX.Config/DataX.Config.Test/Resource/flattenerConfig.json @@ -6,55 +6,71 @@ "input": { "type": "object", "namespace": "input.default", - "fields": { - "blobSchemaFile": "blobschemafile", - "sourceIdRegex": "sourceidregex", - "blobPathRegex": "blobpathregex", - "fileTimeRegex": "filetimeregex", - "fileTimeFormat": "filetimeformat", - "eventhub": { - "type": "object", - "namespace": "eventhub", - "fields": { - "connectionString": "connectionstring", - "consumerGroup": "consumergroup", - "checkpointDir": "checkpointdir", - "checkpointInterval": "checkpointinterval", - "maxRate": "maxrate", - "flushExistingCheckpoints": "flushexistingcheckpoints" - } - }, - "streaming": { - "type": "object", - "namespace": "streaming", - "fields": { - "checkpointDir": "checkpointdir", - "intervalInSeconds": "intervalinseconds" - } - }, - "sources": { - "type": "map", - "namespace": "source", - "fields": { - "target": "target", - "catalogPrefix": "catalogprefix" - } - }, - "referenceData": { - "type": "array", - "namespace": "referencedata", - "element": { - "type": "scopedObject", - "namespaceField": "name", - "fields": { - "path": "path", - "format": "format", - "header": "header", - "delimiter": "delimiter" - } - } + "fields": { + "blobSchemaFile": "blobschemafile", + "sourceIdRegex": "sourceidregex", + "blobPathRegex": "blobpathregex", + "fileTimeRegex": "filetimeregex", + "fileTimeFormat": "filetimeformat", + "eventhub": { + "type": "object", + "namespace": "eventhub", + "fields": { + "connectionString": "connectionstring", + "consumerGroup": "consumergroup", + "checkpointDir": "checkpointdir", + "checkpointInterval": "checkpointinterval", + "maxRate": "maxrate", + "flushExistingCheckpoints": "flushexistingcheckpoints" + } + }, + "blob": { + "type": "array", + "namespace": "blob", + "element": { + "type": "scopedObject", + "namespaceField": "name", + "fields": { + "path": "path", + "format": "format", + "compressiontype": "compressiontype", + "processstarttime": "processstarttime", + "processendtime": "processendtime", + "partitionincrement": "partitionincrement" + } + } + }, + "streaming": { + "type": "object", + "namespace": "streaming", + "fields": { + "checkpointDir": "checkpointdir", + "intervalInSeconds": "intervalinseconds" + } + }, + "sources": { + "type": "map", + "namespace": "source", + "fields": { + "target": "target", + "catalogPrefix": "catalogprefix" + } + }, + "referenceData": { + "type": "array", + "namespace": "referencedata", + "element": { + "type": "scopedObject", + "namespaceField": "name", + "fields": { + "path": "path", + "format": "format", + "header": "header", + "delimiter": "delimiter" } + } } + } }, "process": { "type": "object", diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/flowSaved.json b/Services/DataX.Config/DataX.Config.Test/Resource/flowSaved.json index a2a9d0362..01cb38b18 100644 --- a/Services/DataX.Config/DataX.Config.Test/Resource/flowSaved.json +++ b/Services/DataX.Config/DataX.Config.Test/Resource/flowSaved.json @@ -1,13 +1,14 @@ { - "name": "configgentest", - "icon": "/img/iot.png", - "displayName": "configgentest", - "properties": { - "foo": "bar" - }, + "name": "configgentest", + "icon": "/img/iot.png", + "displayName": "configgentest", + "properties": { + "foo": "bar" + }, "gui": { "name": "configgentest", "displayName": "configgentest", + "databricksToken": null, "owner": "data-accelerator@microsoft.com", "input": { "mode": "streaming", @@ -37,7 +38,8 @@ }, "typeDisplay": "CSV/TSV File" } - ] + ], + "batch": null }, "process": { "timestampColumn": "eventTime", @@ -86,7 +88,10 @@ ], "jobconfig": { "jobNumExecutors": "6", - "jobExecutorMemory": "8000" + "jobExecutorMemory": "8000", + "jobDatabricksAutoScale": false, + "jobDatabricksMinWorkers": null, + "jobDatabricksMaxWorkers": null } }, "outputs": [ @@ -289,6 +294,7 @@ } } ], + "batchList": null, "subscription": null }, "commonProcessor": { @@ -344,59 +350,59 @@ } ] }, - "metrics": { - "sources": [ - { - "name": "events", - "input": { - "type": "MetricApi", - "metricKeys": [ - "DATAX-${name}:Input_DataXProcessedInput_Events_Count" - ] - }, - "output": { - "type": "SumWithTimeChart", - "data": { - "sum": true, - "timechart": true, - "average": true, - "speed": true - } - } - } - ], - "widgets": [ - { - "name": "eventsChart", - "displayName": "Events/Second", - "data": "events_timechart", - "position": "TimeCharts", - "type": "StackAreaChart" - }, - { - "name": "totalEvents", - "displayName": "Events Ingested Today", - "data": "events_sum", - "formatter": "longint", - "type": "SimpleBox", - "position": "FirstRow" - }, - { - "name": "averageEvents", - "displayName": "Avg. Events/Minute", - "data": "events_average", - "formatter": "longint", - "type": "SimpleBox", - "position": "FirstRow" - } - ], - "initParameters": { - "widgetSets": [ - "direct" - ], - "jobNames": { - "type": "getCPSparkJobNames" - } + "metrics": { + "sources": [ + { + "name": "events", + "input": { + "type": "MetricApi", + "metricKeys": [ + "DATAX-${name}:Input_DataXProcessedInput_Events_Count" + ] + }, + "output": { + "type": "SumWithTimeChart", + "data": { + "sum": true, + "timechart": true, + "average": true, + "speed": true + } } + } + ], + "widgets": [ + { + "name": "eventsChart", + "displayName": "Events/Second", + "data": "events_timechart", + "position": "TimeCharts", + "type": "StackAreaChart" + }, + { + "name": "totalEvents", + "displayName": "Events Ingested Today", + "data": "events_sum", + "formatter": "longint", + "type": "SimpleBox", + "position": "FirstRow" + }, + { + "name": "averageEvents", + "displayName": "Avg. Events/Minute", + "data": "events_average", + "formatter": "longint", + "type": "SimpleBox", + "position": "FirstRow" + } + ], + "initParameters": { + "widgetSets": [ + "direct" + ], + "jobNames": { + "type": "getCPSparkJobNames" + } } + } } \ No newline at end of file diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/flowStarted.json b/Services/DataX.Config/DataX.Config.Test/Resource/flowStarted.json index 2958cb9df..8994e2cdb 100644 --- a/Services/DataX.Config/DataX.Config.Test/Resource/flowStarted.json +++ b/Services/DataX.Config/DataX.Config.Test/Resource/flowStarted.json @@ -8,6 +8,7 @@ "gui": { "name": "configgentest", "displayName": "configgentest", + "databricksToken": null, "owner": "data-accelerator@microsoft.com", "input": { "mode": "streaming", @@ -37,7 +38,8 @@ }, "typeDisplay": "CSV/TSV File" } - ] + ], + "batch": null }, "process": { "timestampColumn": "eventTime", @@ -86,7 +88,10 @@ ], "jobconfig": { "jobNumExecutors": "6", - "jobExecutorMemory": "8000" + "jobExecutorMemory": "8000", + "jobDatabricksAutoScale": false, + "jobDatabricksMinWorkers": null, + "jobDatabricksMaxWorkers": null } }, "outputs": [ @@ -289,6 +294,7 @@ } } ], + "batchList": "", "subscription": null }, "commonProcessor": { diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/jobConfig.conf b/Services/DataX.Config/DataX.Config.Test/Resource/jobConfig.conf index 8486e3bbe..b71463ccf 100644 --- a/Services/DataX.Config/DataX.Config.Test/Resource/jobConfig.conf +++ b/Services/DataX.Config/DataX.Config.Test/Resource/jobConfig.conf @@ -50,7 +50,7 @@ datax.job.process.metric.eventhub=keyvault://somekeyvault/metric-eventhubconnect # Output settings # ---------------------------------------------------------------------------- datax.job.output.SimpleRulesTable.blob.compressiontype=none -datax.job.output.SimpleRulesTable.blob.group.main.folder=keyvault://somekeyvault/configgentest-output-28E0EC24007075915D6909C001EA72BF +datax.job.output.SimpleRulesTable.blob.group.main.folder=keyvault://somekeyvault/configgentest-output-D7B7107DC4C5A693C746B31988180C1E datax.job.output.SimpleRulesTable.cosmosdb.connectionstring=keyvault://somekeyvault/configgentest-output-161E93325EAA303AD3816687341330A6 datax.job.output.SimpleRulesTable.cosmosdb.database=Outputs datax.job.output.SimpleRulesTable.cosmosdb.collection=tests diff --git a/Services/DataX.Config/DataX.Config.Test/Resource/jobConfig.json b/Services/DataX.Config/DataX.Config.Test/Resource/jobConfig.json index 441aceecb..737fb036e 100644 --- a/Services/DataX.Config/DataX.Config.Test/Resource/jobConfig.json +++ b/Services/DataX.Config/DataX.Config.Test/Resource/jobConfig.json @@ -87,7 +87,7 @@ "blob": { "groups": { "main": { - "folder": "keyvault://somekeyvault/configgentest-output-28E0EC24007075915D6909C001EA72BF" + "folder": "keyvault://somekeyvault/configgentest-output-D7B7107DC4C5A693C746B31988180C1E" } }, "compressionType": "none", diff --git a/Services/DataX.Config/DataX.Config.Test/RuntimeConfigGenerationBatchTest.cs b/Services/DataX.Config/DataX.Config.Test/RuntimeConfigGenerationBatchTest.cs new file mode 100644 index 000000000..039be5217 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/RuntimeConfigGenerationBatchTest.cs @@ -0,0 +1,191 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Microsoft.VisualStudio.TestTools.UnitTesting; +using DataX.Config.ConfigDataModel; +using DataX.Config.ConfigGeneration.Processor; +using DataX.Config.Test.Extension; +using DataX.Config.Test.Mock; +using DataX.Config.Test.Utility.Mock; +using DataX.Config.Utility; +using System; +using System.Collections.Generic; +using System.Composition; +using System.Composition.Hosting; +using System.IO; +using System.Linq; +using System.Reflection; +using System.Threading.Tasks; +using System.Text.RegularExpressions; + +namespace DataX.Config.Test +{ + [TestClass] + public class RuntimeConfigGenerationBatchTest + { + [ClassInitialize] + public static void Initialize(TestContext tc) + { + InitialConfiguration.Set(Constants.ConfigSettingName_ClusterName, "somecluster"); + InitialConfiguration.Set(Constants.ConfigSettingName_ServiceKeyVaultName, "someservicekeyvault"); + InitialConfiguration.Set(Constants.ConfigSettingName_RuntimeKeyVaultName, "somekeyvault"); + InitialConfiguration.Set(Constants.ConfigSettingName_MetricEventHubConnectionKey, "metric-eventhubconnectionstring"); + + var conf = new ContainerConfiguration() + .WithAssembly(typeof(ConfigGenConfiguration).Assembly) + .WithAssembly(typeof(MockBase).Assembly) + .WithAssembly(Assembly.GetExecutingAssembly()) + .WithProvider(new LoggerAndInstanceExportDescriptorProvider(null, new LoggerFactory())); + + CompositionHost = conf.CreateContainer(); + } + + [ClassCleanup] + public static void Cleanup() + { + if (CompositionHost != null) + { + CompositionHost.Dispose(); + CompositionHost = null; + } + + InitialConfiguration.Clear(); + } + + public RuntimeConfigGenerationBatchTest() + { + CompositionHost.SatisfyImports(this); + } + + private static CompositionHost CompositionHost { get; set; } + + [Import] + private RuntimeConfigGeneration RuntimeConfigGeneration { get; set; } + + [Import] + private DesignTimeStorage DesignTimeStorage { get; set; } + + [Import] + private RuntimeStorage RuntimeStorage { get; set; } + + [Import] + private ICommonDataManager CommonData { get; set; } + + [Import] + private ConfigurationProvider ConfigurationProvider { get; set; } + + [Shared] + [Export(typeof(IFlowDeploymentProcessor))] + private class VerifyJsonConfigGenerated : ProcessorBase + { + public override int GetOrder() + { + // set an order number to be placed right after the GenerateJobConfig processor + return 601; + } + + public override async Task Process(FlowDeploymentSession flowToDeploy) + { + var guiConfig = flowToDeploy.Config.GetGuiConfig(); + if (guiConfig?.Input?.Mode == Constants.InputMode_Batching) + { + var jobConfigs = flowToDeploy.GetJobs().First().JobConfigs; + var reConfigs = jobConfigs.Where(j => !j.IsOneTime).ToList(); + var otConfigs = jobConfigs.Where(j => j.IsOneTime).ToList(); + + Assert.AreEqual(expected: 2, actual: reConfigs.Count); + Assert.AreEqual(expected: 3, actual: otConfigs.Count); + + var startTimeConfig = (DateTime)guiConfig.BatchList[0].Properties.StartTime; + var curTime = startTimeConfig.AddDays(1); + var normalizedStartTime = startTimeConfig.Add(-startTimeConfig.TimeOfDay); + + TimeSpan interval = new TimeSpan(1, 0, 0, 0); + var expected = normalizedStartTime; + foreach(var c in reConfigs) + { + var actualStart = DateTime.Parse(c.ProcessStartTime).ToUniversalTime(); + var actualEnd = DateTime.Parse(c.ProcessEndTime).ToUniversalTime(); + var window = (actualEnd - actualStart).TotalSeconds; + Assert.AreEqual(expected: expected.AddDays(-2), actual: actualStart, message: $"StartTime:{actualStart}"); + Assert.AreEqual(expected: expected.AddDays(1).AddSeconds(-1), actual: actualEnd, message: $"EndTime:{actualEnd}"); + Assert.AreEqual(expected: 259199, actual: window, message: $"Window:{window}"); + Assert.IsTrue(actualStart < curTime); + expected = expected.Add(interval); + } + + var lastScheduleTime = DateTime.Parse(reConfigs.Last().ProcessStartTime).ToUniversalTime(); + Assert.IsTrue(lastScheduleTime < curTime); + + expected = normalizedStartTime; + foreach (var c in otConfigs) + { + var actualStart = DateTime.Parse(c.ProcessStartTime).ToUniversalTime(); + var actualEnd = DateTime.Parse(c.ProcessEndTime).ToUniversalTime(); + var window = (actualEnd - actualStart).TotalSeconds; + Assert.AreEqual(expected: expected, actual: actualStart, message: $"StartTime:{actualStart}"); + Assert.AreEqual(expected: expected.AddDays(1).AddSeconds(-1), actual: actualEnd, message: $"EndTime:{actualEnd}"); + Assert.AreEqual(expected: 86399, actual: window, message: $"Window:{window}"); + expected = expected.Add(interval); + } + + lastScheduleTime = DateTime.Parse(otConfigs.Last().ProcessStartTime).ToUniversalTime(); + Assert.IsTrue(lastScheduleTime > curTime); + } + + return "done"; + } + } + [TestMethod] + public async Task TestBatchSchedule() + { + var flowName = "configgenbatchtest"; + + var testingConfig = await File.ReadAllTextAsync(@"Resource\batchFlow.json"); + var current = DateTime.UtcNow; + + var startTimeConfig = current.AddDays(-1); + var endTimeConfig = current.AddDays(1); + var normalizedStartTime = startTimeConfig.Add(-startTimeConfig.TimeOfDay); + + testingConfig = testingConfig.Replace("${startTime}", startTimeConfig.ToString("o")); + testingConfig = testingConfig.Replace("${endTime}", endTimeConfig.ToString("o")); + + await DesignTimeStorage.SaveByName(flowName, testingConfig, FlowDataManager.DataCollectionName); + + await CommonData.Add("defaultJobTemplate", @"Resource\batchSparkJobTemplate.json"); + await CommonData.Add(ConfigFlattenerManager.DefaultConfigName, @"Resource\flattenerConfig.json"); + await CommonData.Add(FlowDataManager.CommonDataName_DefaultFlowConfig, @"Resource\batchFlowDefault.json"); + + var result = await this.RuntimeConfigGeneration.GenerateRuntimeConfigs(flowName); + + var runtimeConfigFolder = result.Properties?.GetValueOrDefault(PrepareJobConfigVariables.ResultPropertyName_RuntimeConfigFolder, null); + + Assert.IsTrue(result.IsSuccess); + Assert.AreEqual(expected: 8, actual: RuntimeStorage.Cache.Count); + + var processTime = Regex.Replace(normalizedStartTime.ToString("s"), "[^0-9]", ""); + + var actualConf = PropertiesDictionary.From(this.RuntimeStorage.Cache[ResourcePathUtil.Combine(runtimeConfigFolder.ToString() + $@"/Recurring/{processTime}", $"{flowName}.conf")]); + var conf = await File.ReadAllTextAsync(@"Resource\batchJobConfig.conf"); + conf = conf.Replace("${processTime}", processTime); + conf = conf.Replace("${startTime}", normalizedStartTime.AddDays(-2).ToString("o")); + conf = conf.Replace("${endTime}", normalizedStartTime.AddDays(1).AddSeconds(-1).ToString("o")); + var expectedConf = PropertiesDictionary.From(conf); + + var matches = PropertiesDictionary.Match(expectedConf, actualConf).ToList(); + foreach (var match in matches) + { + Console.WriteLine($"prop:{match.Item1 ?? "null"}, expected:<{match.Item2 ?? "null"}>, actual:<{match.Item3 ?? "null"}>"); + } + + foreach (var match in matches) + { + Assert.AreEqual(expected: match.Item2, actual: match.Item3, message: $"property:{match.Item1}"); + } + + Cleanup(); + } + } +} diff --git a/Services/DataX.Config/DataX.Config.Test/RuntimeConfigGenerationTest.cs b/Services/DataX.Config/DataX.Config.Test/RuntimeConfigGenerationTest.cs index a9cc53eff..abb8c66b3 100644 --- a/Services/DataX.Config/DataX.Config.Test/RuntimeConfigGenerationTest.cs +++ b/Services/DataX.Config/DataX.Config.Test/RuntimeConfigGenerationTest.cs @@ -34,7 +34,8 @@ public static void Initialize(TestContext tc) var conf = new ContainerConfiguration() .WithAssembly(typeof(ConfigGenConfiguration).Assembly) .WithAssembly(typeof(MockBase).Assembly) - .WithAssembly(Assembly.GetExecutingAssembly()); + .WithAssembly(Assembly.GetExecutingAssembly()) + .WithProvider(new LoggerAndInstanceExportDescriptorProvider(null, new LoggerFactory())); CompositionHost = conf.CreateContainer(); } @@ -85,16 +86,18 @@ public override int GetOrder() public override async Task Process(FlowDeploymentSession flowToDeploy) { - var expectedConfigContent = await File.ReadAllTextAsync(@"Resource\jobConfig.json"); - var expectedJson = JsonConfig.From(expectedConfigContent); - var actualContentContent = flowToDeploy.GetJobs().First().GetTokenString(GenerateJobConfig.TokenName_JobConfigContent); - var actualJson = JsonConfig.From(actualContentContent); - - foreach (var match in JsonConfig.Match(expectedJson, actualJson)) + if(flowToDeploy.Name == "configgentest") { - Assert.AreEqual(expected: match.Item2, actual: match.Item3, message: $"path:{match.Item1}"); + var expectedConfigContent = await File.ReadAllTextAsync(@"Resource\jobConfig.json"); + var expectedJson = JsonConfig.From(expectedConfigContent); + var actualContentContent = flowToDeploy.GetJobs().First().GetTokenString(GenerateJobConfig.TokenName_JobConfigContent); + var actualJson = JsonConfig.From(actualContentContent); + + foreach (var match in JsonConfig.Match(expectedJson, actualJson)) + { + Assert.AreEqual(expected: match.Item2, actual: match.Item3, message: $"path:{match.Item1}"); + } } - return "done"; } } @@ -158,6 +161,8 @@ public async Task TestEndToEndGeneration() { Assert.AreEqual(expected: match.Item2, actual: match.Item3, message: $"path:{match.Item1}"); } + + Cleanup(); } } } diff --git a/Services/DataX.Config/DataX.Config.Test/SparkJobOperationTest.cs b/Services/DataX.Config/DataX.Config.Test/SparkJobOperationTest.cs new file mode 100644 index 000000000..c940b8b54 --- /dev/null +++ b/Services/DataX.Config/DataX.Config.Test/SparkJobOperationTest.cs @@ -0,0 +1,37 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Microsoft.VisualStudio.TestTools.UnitTesting; + +namespace DataX.Config.Test +{ + [TestClass] + public class SparkJobOperationTest + { + [TestMethod] + public void TestVerifyJobStoppedt() + { + JobState errorState = JobState.Error; + JobState idleState = JobState.Idle; + JobState runningState = JobState.Running; + JobState startingState = JobState.Starting; + JobState successState = JobState.Success; + + //Verify error state is considered that the job is not running + Assert.IsTrue(SparkJobOperation.VerifyJobStopped(errorState)); + + //Verify idle state is considered that the job is not running + Assert.IsTrue(SparkJobOperation.VerifyJobStopped(idleState)); + + //Verify running state is considered that the job is running + Assert.IsFalse(SparkJobOperation.VerifyJobStopped(runningState)); + + //Verify starting state is considered that the job is running + Assert.IsFalse(SparkJobOperation.VerifyJobStopped(startingState)); + + //Verify success state is considered that the job is not running + Assert.IsTrue(SparkJobOperation.VerifyJobStopped(successState)); + } + } +} diff --git a/Services/DataX.Config/DataX.Config.sln b/Services/DataX.Config/DataX.Config.sln index 94d186dbc..cff3ac9ef 100644 --- a/Services/DataX.Config/DataX.Config.sln +++ b/Services/DataX.Config/DataX.Config.sln @@ -65,6 +65,10 @@ Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Local.Test", " EndProject Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.ServiceHost.ServiceFabric", "..\DataX.ServiceHost\DataX.ServiceHost.ServiceFabric\DataX.ServiceHost.ServiceFabric.csproj", "{A5627F57-73C6-4E81-B36A-B91EDCC03487}" EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.DatabricksClient", "DataX.Config.DatabricksClient\DataX.Config.DatabricksClient.csproj", "{8A0240B7-397A-4F3D-8433-2AF694E9D399}" +EndProject +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "DataX.Config.DatabricksClient.Test", "DataX.Config.DatabricksClient.Test\DataX.Config.DatabricksClient.Test.csproj", "{51D7A7A5-F243-4118-91F7-C4C6E3E3DB99}" +EndProject Global GlobalSection(SolutionConfigurationPlatforms) = preSolution Debug|Any CPU = Debug|Any CPU @@ -151,6 +155,14 @@ Global {A5627F57-73C6-4E81-B36A-B91EDCC03487}.Debug|Any CPU.Build.0 = Debug|Any CPU {A5627F57-73C6-4E81-B36A-B91EDCC03487}.Release|Any CPU.ActiveCfg = Release|Any CPU {A5627F57-73C6-4E81-B36A-B91EDCC03487}.Release|Any CPU.Build.0 = Release|Any CPU + {8A0240B7-397A-4F3D-8433-2AF694E9D399}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {8A0240B7-397A-4F3D-8433-2AF694E9D399}.Debug|Any CPU.Build.0 = Debug|Any CPU + {8A0240B7-397A-4F3D-8433-2AF694E9D399}.Release|Any CPU.ActiveCfg = Release|Any CPU + {8A0240B7-397A-4F3D-8433-2AF694E9D399}.Release|Any CPU.Build.0 = Release|Any CPU + {51D7A7A5-F243-4118-91F7-C4C6E3E3DB99}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {51D7A7A5-F243-4118-91F7-C4C6E3E3DB99}.Debug|Any CPU.Build.0 = Debug|Any CPU + {51D7A7A5-F243-4118-91F7-C4C6E3E3DB99}.Release|Any CPU.ActiveCfg = Release|Any CPU + {51D7A7A5-F243-4118-91F7-C4C6E3E3DB99}.Release|Any CPU.Build.0 = Release|Any CPU EndGlobalSection GlobalSection(SolutionProperties) = preSolution HideSolutionNode = FALSE @@ -171,6 +183,8 @@ Global {8311D49B-3333-45E8-853C-B5790C2317FD} = {0D3EE049-5C0E-415C-81F1-6F19C96BF51F} {D5454F8C-E31D-4731-94D9-C2E26E9DB447} = {0D3EE049-5C0E-415C-81F1-6F19C96BF51F} {4EC08127-ED3A-45C4-A9F9-15E154C1ED4A} = {327F5A7B-C05A-4A0A-9E8D-6DF2D059BD80} + {8A0240B7-397A-4F3D-8433-2AF694E9D399} = {327F5A7B-C05A-4A0A-9E8D-6DF2D059BD80} + {51D7A7A5-F243-4118-91F7-C4C6E3E3DB99} = {327F5A7B-C05A-4A0A-9E8D-6DF2D059BD80} EndGlobalSection GlobalSection(ExtensibilityGlobals) = postSolution SolutionGuid = {3D094544-AA70-412B-979A-06653F810C28} diff --git a/Services/DataX.Config/DataX.Config/Client/IKeyVaultClient.cs b/Services/DataX.Config/DataX.Config/Client/IKeyVaultClient.cs index be31c90d2..9ed7b5d08 100644 --- a/Services/DataX.Config/DataX.Config/Client/IKeyVaultClient.cs +++ b/Services/DataX.Config/DataX.Config/Client/IKeyVaultClient.cs @@ -22,15 +22,31 @@ public interface IKeyVaultClient /// name of the specified secret /// value of the secret or throw Task GetSecretFromKeyVaultAsync(string keyvaultName, string secretName); - + /// /// Save the secret and return the secret uri /// /// keyvault to save the secret /// name of the secret /// value of the secret + /// value of the uriPrefix /// specify whether the generated secret uri has a hashed suffix, by default it is false /// secret id - Task SaveSecretAsync(string keyvaultName, string secretName, string secretValue, bool hashSuffix = false); + Task SaveSecretAsync(string keyvaultName, string secretName, string secretValue, string sparkType, bool hashSuffix = false); + + /// + /// Save the secret and return the secret uri + /// + /// Uri of the secret + /// value of the secret + /// secret id + Task SaveSecretAsync(string secretUri, string secretValue); + + /// + /// Generate a secret uri prefix based on sparkType + /// + /// spark type + /// + string GetUriPrefix(string sparkType); } } diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/Constants.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/Constants.cs index 0e75d5233..a40353229 100644 --- a/Services/DataX.Config/DataX.Config/ConfigDataModel/Constants.cs +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/Constants.cs @@ -1,39 +1,62 @@ -// ********************************************************************* -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License -// ********************************************************************* -using System; -using System.Collections.Generic; -using System.Text; +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using System; +using System.Collections.Generic; +using System.Text; + +namespace DataX.Config.ConfigDataModel +{ + public static class Constants + { + public const string ConfigSettingName_RuntimeKeyVaultName = "sparkKeyVaultName"; + public const string ConfigSettingName_ServiceKeyVaultName = "serviceKeyVaultName"; + public const string ConfigSettingName_MetricEventHubConnectionKey = "metricEventHubConnectionStringKey"; + public const string ConfigSettingName_ConfigFolderHost = "cpConfigFolderBase"; + public const string ConfigSettingName_ConfigFolderContainerPath = "containerPath"; + public const string ConfigSettingName_ConfigGenSubscriptionId = "configgen-subscriptionid"; + public const string ConfigSettingName_ConfigGenClientId = "configgenClientId"; + public const string ConfigSettingName_ConfigGenTenantId = "configgenTenantId"; + public const string ConfigSettingName_ClusterName = "sparkClusterName"; + public const string ConfigSettingName_RuntimeApplicationInsightKey = "applicationInsightKey"; + public const string ConfigSettingName_EnableOneBox = "enableOneBox"; + public const string ConfigSettingName_LocalRoot = "localRoot"; + public const string ConfigSettingName_LocalMetricsHttpEndpoint = "localMetricsHttpEndpoint"; + public const string ConfigSettingName_SecretPrefix = "configgenSecretPrefix"; + public const string ConfigSettingName_ResourceCreation = "resourceCreation"; + public const string ConfigSettingName_CosmosDBConfigConnectionString = "CosmosDBConfigConnectionString"; + public const string ConfigSettingName_CosmosDBConfigDatabaseName = "CosmosDBConfigDatabaseName"; + public const string ConfigSettingName_CosmosDBConfigCollectionName = "CosmosDBConfigCollectionName"; + public const string ConfigSettingName_AppInsightsIntrumentationKey = "AppInsightsIntrumentationKey"; + public const string ConfigSettingName_SparkType = "sparkType"; + + public const string TokenName_SparkJobConfigFilePath = "sparkJobConfigFilePath"; + public const string TokenName_SparkJobName = "sparkJobName"; + + public const string ConfigProcessorResultStatus_Completed = "done"; + + public const string InputType_Kafka = "kafka"; + public const string InputType_IoTHub = "iothub"; + public const string InputType_EventHub = "events"; + public const string InputType_KafkaEventHub = "kafkaeventhub"; + public const string InputType_Blob = "blob"; + + public const string InputMode_Streaming = "streaming"; + public const string InputMode_Batching = "batching"; + + public const string Batch_Recurring = "recurring"; + public const string Batch_OneTime = "oneTime"; + + public const string SparkTypeDataBricks = "databricks"; + public const string SparkTypeHDInsight = "HDInsight"; -namespace DataX.Config.ConfigDataModel -{ - public static class Constants - { - public const string ConfigSettingName_RuntimeKeyVaultName = "sparkKeyVaultName"; - public const string ConfigSettingName_ServiceKeyVaultName= "serviceKeyVaultName"; - public const string ConfigSettingName_MetricEventHubConnectionKey = "metricEventHubConnectionStringKey"; - public const string ConfigSettingName_ConfigFolderHost = "cpConfigFolderBase"; - public const string ConfigSettingName_ConfigFolderContainerPath = "containerPath"; - public const string ConfigSettingName_ConfigGenSubscriptionId = "configgen-subscriptionid"; - public const string ConfigSettingName_ConfigGenClientId = "configgenClientId"; - public const string ConfigSettingName_ConfigGenTenantId = "configgenTenantId"; - public const string ConfigSettingName_ClusterName = "sparkClusterName"; - public const string ConfigSettingName_RuntimeApplicationInsightKey = "applicationInsightKey"; - public const string ConfigSettingName_EnableOneBox = "enableOneBox"; - public const string ConfigSettingName_LocalRoot = "localRoot"; - public const string ConfigSettingName_LocalMetricsHttpEndpoint = "localMetricsHttpEndpoint"; - public const string ConfigSettingName_SecretPrefix = "configgenSecretPrefix"; - public const string ConfigSettingName_ResourceCreation = "resourceCreation"; + public const string PrefixSecretScope = "secretscope"; + public const string PrefixKeyVault = "keyvault"; + public const string PrefixHdfs = "hdfs://"; + public const string PrefixDbfs = "dbfs:/"; + public const string PrefixDbfsMount = "mnt/livequery/"; - public const string TokenName_SparkJobConfigFilePath = "sparkJobConfigFilePath"; - public const string TokenName_SparkJobName = "sparkJobName"; - - public const string ConfigProcessorResultStatus_Completed = "done"; - - public const string InputType_Kafka = "kafka"; - public const string InputType_IoTHub = "iothub"; - public const string InputType_EventHub = "events"; - public const string InputType_KafkaEventHub = "kafkaeventhub"; - } -} + public const string AccountSecretPrefix = "datax-sa-"; + } +} diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/Flattener/FlattenerMappingObject.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/Flattener/FlattenerMappingObject.cs index 405721951..025e49aa5 100644 --- a/Services/DataX.Config/DataX.Config/ConfigDataModel/Flattener/FlattenerMappingObject.cs +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/Flattener/FlattenerMappingObject.cs @@ -23,6 +23,12 @@ public override IEnumerable> FlattenJToken(JToken jt) { var fieldValue = field.Value; var subValue = jt[field.Key]; + + if (subValue?.Type == JTokenType.Date) + { + subValue = subValue.Value().ToString("o"); + } + switch (fieldValue.Type) { case JTokenType.Object: diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiConfig.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiConfig.cs index 1b74fe52a..5305ad8d5 100644 --- a/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiConfig.cs +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiConfig.cs @@ -28,6 +28,8 @@ public static FlowGuiConfig From(string json) public string Name { get; set; } [JsonProperty("displayName")] public string DisplayName { get; set; } + [JsonProperty("databricksToken")] + public string DatabricksToken { get; set; } [JsonProperty("owner")] public string Owner { get; set; } [JsonProperty("input")] @@ -40,6 +42,8 @@ public static FlowGuiConfig From(string json) public FlowGuiOutputTemplate[] OutputTemplates { get; set; } [JsonProperty("rules")] public FlowGuiRule[] Rules { get; set; } + [JsonProperty("batchList")] + public FlowGuiInputBatchJob[] BatchList { get; set; } [JsonProperty("subscription")] public string Subscription { get; set; } #endregion diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInput.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInput.cs index 2eb886276..a04423e84 100644 --- a/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInput.cs +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInput.cs @@ -19,5 +19,8 @@ public class FlowGuiInput [JsonProperty("referenceData")] public FlowGuiReferenceData[] ReferenceData { get; set; } + + [JsonProperty("batch")] + public FlowGuiInputBatchInput[] Batch { get; set; } } } diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInputBatchInput.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInputBatchInput.cs new file mode 100644 index 000000000..138ba6742 --- /dev/null +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInputBatchInput.cs @@ -0,0 +1,17 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Newtonsoft.Json; + +namespace DataX.Config.ConfigDataModel +{ + public class FlowGuiInputBatchInput + { + [JsonProperty("type")] + public string Type { get; set; } + + [JsonProperty("properties")] + public FlowGuiInputBatchInputProperties Properties { get; set; } + } +} diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInputBatchInputProperties.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInputBatchInputProperties.cs new file mode 100644 index 000000000..c4ec62ef1 --- /dev/null +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInputBatchInputProperties.cs @@ -0,0 +1,23 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Newtonsoft.Json; + +namespace DataX.Config.ConfigDataModel +{ + public class FlowGuiInputBatchInputProperties + { + [JsonProperty("connection")] + public string Connection { get; set; } + + [JsonProperty("path")] + public string Path { get; set; } + + [JsonProperty("formatType")] + public string FormatType { get; set; } + + [JsonProperty("compressionType")] + public string CompressionType { get; set; } + } +} diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInputBatchJob.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInputBatchJob.cs new file mode 100644 index 000000000..d7695f52f --- /dev/null +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInputBatchJob.cs @@ -0,0 +1,23 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Newtonsoft.Json; + +namespace DataX.Config.ConfigDataModel +{ + public class FlowGuiInputBatchJob + { + [JsonProperty("id")] + public string Id { get; set; } + + [JsonProperty("type")] + public string Type { get; set; } + + [JsonProperty("properties")] + public FlowGuiInputBatchJobProperties Properties { get; set; } + + [JsonProperty("disabled")] + public bool Disabled { get; set; } + } +} diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInputBatchJobProperties.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInputBatchJobProperties.cs new file mode 100644 index 000000000..4fe9cbb1a --- /dev/null +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiInputBatchJobProperties.cs @@ -0,0 +1,39 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Newtonsoft.Json; +using System; + +namespace DataX.Config.ConfigDataModel +{ + public class FlowGuiInputBatchJobProperties + { + [JsonProperty("interval")] + public string Interval { get; set; } + + [JsonProperty("intervalType")] + public string IntervalType { get; set; } + + [JsonProperty("delay")] + public string Delay { get; set; } + + [JsonProperty("delayType")] + public string DelayType { get; set; } + + [JsonProperty("window")] + public string Window { get; set; } + + [JsonProperty("windowType")] + public string WindowType { get; set; } + + [JsonProperty("startTime")] + public DateTime? StartTime { get; set; } + + [JsonProperty("endTime")] + public DateTime? EndTime { get; set; } + + [JsonProperty("lastProcessedTime")] + public string LastProcessedTime { get; set; } + } +} diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiJobConfig.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiJobConfig.cs index e0eab3630..d200a019f 100644 --- a/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiJobConfig.cs +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiJobConfig.cs @@ -13,5 +13,14 @@ public class FlowGuiJobConfig [JsonProperty("jobExecutorMemory")] public string JobExecutorMemory { get; set; } + + [JsonProperty("jobDatabricksAutoScale")] + public bool JobDatabricksAutoScale { get; set; } + + [JsonProperty("jobDatabricksMinWorkers")] + public string JobDatabricksMinWorkers { get; set; } + + [JsonProperty("jobDatabricksMaxWorkers")] + public string JobDatabricksMaxWorkers { get; set; } } } diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiOutputProperties.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiOutputProperties.cs index 6248ffe66..6cc323aa8 100644 --- a/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiOutputProperties.cs +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/FlowGui/FlowGuiOutputProperties.cs @@ -6,7 +6,7 @@ namespace DataX.Config.ConfigDataModel { - // Class representing union schema of all supported ouputs (Blob, CosmosDB, EventHub, Metrics) + // Class representing union schema of all supported ouputs (Blob, CosmosDB, EventHub, SQL Server, Metrics) public class FlowGuiOutputProperties { [JsonProperty("connectionString", NullValueHandling = NullValueHandling.Ignore)] @@ -32,5 +32,14 @@ public class FlowGuiOutputProperties [JsonProperty("collection", NullValueHandling = NullValueHandling.Ignore)] public string Collection; + + [JsonProperty("tableName", NullValueHandling = NullValueHandling.Ignore)] + public string TableName; + + [JsonProperty("writeMode", NullValueHandling = NullValueHandling.Ignore)] + public string WriteMode; + + [JsonProperty("useBulkInsert", NullValueHandling = NullValueHandling.Ignore)] + public bool? UseBulkInsert; } } diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/RuntimeConfig/FlowOutputSpec.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/RuntimeConfig/FlowOutputSpec.cs index 0e35d0a8d..168a66829 100644 --- a/Services/DataX.Config/DataX.Config/ConfigDataModel/RuntimeConfig/FlowOutputSpec.cs +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/RuntimeConfig/FlowOutputSpec.cs @@ -26,5 +26,8 @@ public class FlowOutputSpec [JsonProperty("httppost", NullValueHandling = NullValueHandling.Ignore)] public FlowHttpOutputSpec HttpOutput { get; set; } + [JsonProperty("sqlServer", NullValueHandling = NullValueHandling.Ignore)] + public FlowSqlOutputSpec SqlOutput { get; set; } + } } diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/RuntimeConfig/FlowSqlOutputSpec.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/RuntimeConfig/FlowSqlOutputSpec.cs new file mode 100644 index 000000000..213ea0b5b --- /dev/null +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/RuntimeConfig/FlowSqlOutputSpec.cs @@ -0,0 +1,35 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Newtonsoft.Json; + +namespace DataX.Config.ConfigDataModel.RuntimeConfig +{ + public class FlowSqlOutputSpec + { + [JsonProperty("connectionStringRef")] + public string ConnectionStringRef { get; set; } + + [JsonProperty("tableName")] + public string TableName { get; set; } + + [JsonProperty("databaseName")] + public string DatabaseName { get; set; } + + [JsonProperty("user")] + public string User { get; set; } + + [JsonProperty("password")] + public string Password { get; set; } + + [JsonProperty("url")] + public string Url { get; set; } + + [JsonProperty("writeMode")] + public string WriteMode { get; set; } + + [JsonProperty("useBulkInsert")] + public bool? UseBulkInsert { get; set; } + } +} diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/RuntimeConfig/InputBatchingSpec.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/RuntimeConfig/InputBatchingSpec.cs new file mode 100644 index 000000000..af8b5a9a8 --- /dev/null +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/RuntimeConfig/InputBatchingSpec.cs @@ -0,0 +1,33 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Newtonsoft.Json; +using System.Collections.Generic; + +namespace DataX.Config.ConfigDataModel.RuntimeConfig +{ + public class InputBatchingSpec + { + [JsonProperty("name")] + public string Name { get; set; } + + [JsonProperty("path")] + public string Path { get; set; } + + [JsonProperty("format")] + public string Format { get; set; } + + [JsonProperty("compressiontype")] + public string CompressionType { get; set; } + + [JsonProperty("processstarttime")] + public string ProcessStartTime { get; set; } + + [JsonProperty("processendtime")] + public string ProcessEndTime { get; set; } + + [JsonProperty("partitionincrement")] + public string PartitionIncrement { get; set; } + } +} diff --git a/Services/DataX.Config/DataX.Config/ConfigDataModel/SparkJobConfig.cs b/Services/DataX.Config/DataX.Config/ConfigDataModel/SparkJobConfig.cs index be53a68da..68d8c2990 100644 --- a/Services/DataX.Config/DataX.Config/ConfigDataModel/SparkJobConfig.cs +++ b/Services/DataX.Config/DataX.Config/ConfigDataModel/SparkJobConfig.cs @@ -37,6 +37,12 @@ public static SparkJobConfig From(string json) [JsonProperty("cluster")] public string Cluster { get; set; } + /// + /// Path to key vault secret that stores databricks token + /// + [JsonProperty("databricksToken")] + public string DatabricksToken { get; set; } + /// /// options for submitting the job /// diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/JobConfig.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/JobConfig.cs new file mode 100644 index 000000000..24ec80637 --- /dev/null +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/JobConfig.cs @@ -0,0 +1,33 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Config.ConfigDataModel; +using DataX.Config.Templating; +using System; +using System.Collections.Generic; +using System.Text; + +namespace DataX.Config +{ + public class JobConfig + { + public string Name { get; set; } + + public string SparkJobName { get; set; } + + public string Content { get; set; } + + public string FilePath { get; set; } + + public string SparkFilePath { get; set; } + + public string ProcessStartTime { get; set; } + + public string ProcessEndTime { get; set; } + + public string ProcessingTime { get; set; } + + public bool IsOneTime { get; set; } + } +} diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/JobDeploymentSession.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/JobDeploymentSession.cs index 9dd31742f..f3f4dc53d 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/JobDeploymentSession.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/JobDeploymentSession.cs @@ -16,9 +16,12 @@ public JobDeploymentSession(string name, TokenDictionary tokens, FlowDeploymentS : base(name, tokens) { this.Flow= flow; + this.JobConfigs = new List(); } - protected FlowDeploymentSession Flow { get; } + public FlowDeploymentSession Flow { get; } + + public List JobConfigs { get; } public string SparkJobConfigFilePath { diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S200_MergeDefaultConfig.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S200_MergeDefaultConfig.cs index c8431f4db..4e7def579 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S200_MergeDefaultConfig.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S200_MergeDefaultConfig.cs @@ -38,7 +38,7 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) { var flowConfig = flowToDeploy.Config; - var defaultConfig = await FlowData.GetFlowConfigByInputType(flowConfig.GetGuiConfig().Input.InputType, flowToDeploy.Tokens); + var defaultConfig = await FlowData.GetFlowConfigByInputType(flowConfig.GetGuiConfig()?.Input?.InputType, flowToDeploy.Tokens); if (defaultConfig == null) { return "defaultConfig is null, skipped"; diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S450_GenerateTransformFile.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S450_PrepareTransformFile.cs similarity index 69% rename from Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S450_GenerateTransformFile.cs rename to Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S450_PrepareTransformFile.cs index a42b694bc..3fd2524c9 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S450_GenerateTransformFile.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S450_PrepareTransformFile.cs @@ -9,6 +9,8 @@ using System.Composition; using System.Linq; using System.Threading.Tasks; +using DataX.Config.ConfigDataModel; +using DataX.Utility.KeyVault; namespace DataX.Config.ConfigGeneration.Processor { @@ -17,19 +19,21 @@ namespace DataX.Config.ConfigGeneration.Processor /// [Shared] [Export(typeof(IFlowDeploymentProcessor))] - public class GenerateTransformFile : ProcessorBase + public class PrepareTransformFile : ProcessorBase { public const string TokenName_TransformFile = "processTransforms"; public const string AttachmentName_CodeGenObject = "rulesCode"; + private ConfigGenConfiguration Configuration { get; } private IKeyVaultClient KeyVaultClient { get; } private IRuntimeConfigStorage RuntimeStorage { get; } [ImportingConstructor] - public GenerateTransformFile(IKeyVaultClient keyvaultClient, IRuntimeConfigStorage runtimeStorage) + public PrepareTransformFile(IKeyVaultClient keyvaultClient, IRuntimeConfigStorage runtimeStorage, ConfigGenConfiguration conf) { KeyVaultClient = keyvaultClient; RuntimeStorage = runtimeStorage; + Configuration = conf; } public override int GetOrder() @@ -37,7 +41,10 @@ public override int GetOrder() return 450; } - + /// + /// Generate and set the info for the transform file which will be used to generate JobConfig + /// + /// public override async Task Process(FlowDeploymentSession flowToDeploy) { var config = flowToDeploy.Config; @@ -48,7 +55,7 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) } string queries = string.Join("\n", guiConfig.Process?.Queries); - string ruleDefinitions = RuleDefinitionGenerator.GenerateRuleDefinitions(guiConfig.Rules, config.Name); + string ruleDefinitions = RuleDefinitionGenerator.GenerateRuleDefinitions(guiConfig.Rules, config.Name); RulesCode rulesCode = CodeGen.GenerateCode(queries, ruleDefinitions, config.Name); Ensure.NotNull(rulesCode, "rulesCode"); @@ -56,18 +63,16 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) // Save the rulesCode object for downstream processing flowToDeploy.SetAttachment(AttachmentName_CodeGenObject, rulesCode); - var runtimeConfigBaseFolder = flowToDeploy.GetTokenString(PrepareJobConfigVariables.TokenName_RuntimeConfigFolder); - Ensure.NotNull(runtimeConfigBaseFolder, "runtimeConfigBaseFolder"); - var runtimeKeyVaultName = flowToDeploy.GetTokenString(PortConfigurationSettings.TokenName_RuntimeKeyVaultName); Ensure.NotNull(runtimeKeyVaultName, "runtimeKeyVaultName"); - var filePath = ResourcePathUtil.Combine(runtimeConfigBaseFolder, $"{config.Name}-combined.txt"); - var transformFilePath = await RuntimeStorage.SaveFile(filePath, rulesCode.Code); var secretName = $"{config.Name}-transform"; - var transformFileSecret = await KeyVaultClient.SaveSecretAsync(runtimeKeyVaultName, secretName, transformFilePath); + Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string sparkType); + var uriPrefix = KeyVaultClient.GetUriPrefix(sparkType); + var transformFileSecret = SecretUriParser.ComposeUri(runtimeKeyVaultName, secretName, uriPrefix); flowToDeploy.SetStringToken(TokenName_TransformFile, transformFileSecret); + await Task.CompletedTask; return "done"; } } diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_PrepareProjectionFile.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_PrepareProjectionFile.cs new file mode 100644 index 000000000..6e49e74b1 --- /dev/null +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_PrepareProjectionFile.cs @@ -0,0 +1,61 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Config.ConfigDataModel; +using DataX.Config.Utility; +using DataX.Contract; +using DataX.Utility.KeyVault; +using System; +using System.Collections.Generic; +using System.Composition; +using System.Text; +using System.Threading.Tasks; + +namespace DataX.Config.ConfigGeneration.Processor +{ + /// + /// Produce the projection file section + /// + [Shared] + [Export(typeof(IFlowDeploymentProcessor))] + public class PrepareProjectionFile: ProcessorBase + { + public const string TokenName_ProjectionFiles = "processProjections"; + + [ImportingConstructor] + public PrepareProjectionFile(IRuntimeConfigStorage runtimeStorage, IKeyVaultClient keyvaultClient, ConfigGenConfiguration conf) + { + RuntimeStorage = runtimeStorage; + KeyVaultClient = keyvaultClient; + Configuration = conf; + } + + private ConfigGenConfiguration Configuration { get; } + private IRuntimeConfigStorage RuntimeStorage { get; } + private IKeyVaultClient KeyVaultClient { get; } + + /// + /// Generate and set the info for the projection file which will be used to generate JobConfig + /// + /// + public override async Task Process(FlowDeploymentSession flowToDeploy) + { + var config = flowToDeploy.Config; + var runtimeConfigBaseFolder = flowToDeploy.GetTokenString(PrepareJobConfigVariables.TokenName_RuntimeConfigFolder); + Ensure.NotNull(runtimeConfigBaseFolder, "runtimeConfigBaseFolder"); + + var runtimeKeyVaultName = flowToDeploy.GetTokenString(PortConfigurationSettings.TokenName_RuntimeKeyVaultName); + Ensure.NotNull(runtimeKeyVaultName, "runtimeKeyVaultName"); + + var secretName = $"{config.Name}-projectionfile"; + Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string sparkType); + var uriPrefix = KeyVaultClient.GetUriPrefix(sparkType); + var projectionFileSecret = SecretUriParser.ComposeUri(runtimeKeyVaultName, secretName, uriPrefix); + flowToDeploy.SetObjectToken(TokenName_ProjectionFiles, new string[] { projectionFileSecret }); + + await Task.CompletedTask; + return "done"; + } + } +} diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_PrepareSchemaFile.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_PrepareSchemaFile.cs new file mode 100644 index 000000000..2dcbd6c12 --- /dev/null +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_PrepareSchemaFile.cs @@ -0,0 +1,59 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Config.ConfigDataModel; +using DataX.Config.Utility; +using DataX.Contract; +using DataX.Utility.KeyVault; +using System; +using System.Collections.Generic; +using System.Composition; +using System.IO; +using System.Text; +using System.Threading.Tasks; + +namespace DataX.Config.ConfigGeneration.Processor +{ + /// + /// Produce the schema file section + /// + [Shared] + [Export(typeof(IFlowDeploymentProcessor))] + public class PrepareSchemaFile: ProcessorBase + { + public const string TokenName_InputSchemaFilePath = "inputSchemaFilePath"; + + [ImportingConstructor] + public PrepareSchemaFile(IKeyVaultClient keyvaultClient, IRuntimeConfigStorage runtimeStorage, ConfigGenConfiguration conf) + { + KeyVaultClient = keyvaultClient; + RuntimeStorage = runtimeStorage; + Configuration = conf; + } + + private ConfigGenConfiguration Configuration { get; } + private IKeyVaultClient KeyVaultClient { get; } + private IRuntimeConfigStorage RuntimeStorage { get; } + + /// + /// Generate and set the info for the input schema file which will be used to generate JobConfig + /// + /// + public override async Task Process(FlowDeploymentSession flowToDeploy) + { + var config = flowToDeploy.Config; + var runtimeKeyVaultName = flowToDeploy.GetTokenString(PortConfigurationSettings.TokenName_RuntimeKeyVaultName); + Ensure.NotNull(runtimeKeyVaultName, "runtimeKeyVaultName"); + + var secretName = $"{config.Name}-inputschemafile"; + Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string sparkType); + var uriPrefix = KeyVaultClient.GetUriPrefix(sparkType); + var schemaFileSecret = SecretUriParser.ComposeUri(runtimeKeyVaultName, secretName, uriPrefix); + flowToDeploy.SetStringToken(TokenName_InputSchemaFilePath, schemaFileSecret); + + await Task.CompletedTask; + return "done"; + } + } +} diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveAzureFunction.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveAzureFunction.cs index 4e56f1410..734913c92 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveAzureFunction.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveAzureFunction.cs @@ -53,6 +53,7 @@ public override async Task HandleSensitiveData(FlowGuiConfig guiC keyvaultName: RuntimeKeyVaultName.Value, secretName: secretName, secretValue: code, + sparkType: Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string sparkType) ? sparkType : null, hashSuffix: true); func.Properties.Code = secretUri; diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveJarUDF.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveJarUDF.cs index be5f1ee18..567d9effc 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveJarUDF.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveJarUDF.cs @@ -52,6 +52,7 @@ public override async Task HandleSensitiveData(FlowGuiConfig guiC keyvaultName: RuntimeKeyVaultName.Value, secretName: secretName, secretValue: path, + sparkType: Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string sparkType) ? sparkType : null, hashSuffix: true); func.Properties.Path = secretUri; diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveOutputs.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveOutputs.cs index 30e3a527a..d79f87906 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveOutputs.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveOutputs.cs @@ -54,12 +54,12 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) return "no gui input, skipped"; } - var rulesCode = flowToDeploy.GetAttachment(GenerateTransformFile.AttachmentName_CodeGenObject); + var rulesCode = flowToDeploy.GetAttachment(PrepareTransformFile.AttachmentName_CodeGenObject); Ensure.NotNull(rulesCode, "rulesCode"); Ensure.NotNull(rulesCode.MetricsRoot, "rulesCode.MetricsRoot"); Ensure.NotNull(rulesCode.MetricsRoot.metrics, "rulesCode.MetricsRoot.metrics"); - var outputs = await ProcessOutputs(guiConfig.Outputs, rulesCode, config.Name); + var outputs = await ProcessOutputs(guiConfig.Input.Mode, guiConfig.Outputs, rulesCode, config.Name); flowToDeploy.SetObjectToken(TokenName_Outputs, outputs); @@ -67,7 +67,7 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) } public override async Task HandleSensitiveData(FlowGuiConfig guiConfig) - { + { var outputsData = guiConfig?.Outputs; if (outputsData != null && outputsData.Length > 0) { @@ -81,6 +81,7 @@ public override async Task HandleSensitiveData(FlowGuiConfig guiC keyvaultName: RuntimeKeyVaultName.Value, secretName: secretName, secretValue: connStr, + sparkType: Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string sparkType) ? sparkType : null, hashSuffix: true); rd.Properties.ConnectionString = secretUri; @@ -92,7 +93,7 @@ public override async Task HandleSensitiveData(FlowGuiConfig guiC } - private async Task ProcessOutputs(FlowGuiOutput[] uiOutputs, RulesCode rulesCode, string configName) + private async Task ProcessOutputs(string inputMode, FlowGuiOutput[] uiOutputs, RulesCode rulesCode, string configName) { var outputList = uiOutputs.Select(o => o.Id).ToList(); @@ -130,14 +131,14 @@ private async Task ProcessOutputs(FlowGuiOutput[] uiOutputs, R case "cosmosdb": { var cosmosDbOutput = ProcessOutputCosmosDb(output); - Ensure.EnsureNullElseThrowNotSupported(flowOutput.CosmosDbOutput, "Multiple target cosmosDB ouptut for same dataset not supported."); + Ensure.EnsureNullElseThrowNotSupported(flowOutput.CosmosDbOutput, "Multiple target cosmosDB output for same dataset not supported."); flowOutput.CosmosDbOutput = cosmosDbOutput; break; } case "eventhub": { var eventhubOutput = ProcessOutputEventHub(output); - Ensure.EnsureNullElseThrowNotSupported(flowOutput.EventHubOutput, "Multiple target eventHub/metric ouptut for same dataset not supported."); + Ensure.EnsureNullElseThrowNotSupported(flowOutput.EventHubOutput, "Multiple target eventHub/metric output for same dataset not supported."); flowOutput.EventHubOutput = eventhubOutput; break; } @@ -148,33 +149,40 @@ private async Task ProcessOutputs(FlowGuiOutput[] uiOutputs, R if (Configuration.TryGet(Constants.ConfigSettingName_LocalMetricsHttpEndpoint, out string localMetricsEndpoint)) { var httpOutput = ProcessLocalOutputMetric(configName, localMetricsEndpoint); - Ensure.EnsureNullElseThrowNotSupported(flowOutput.HttpOutput, "Multiple target httpost/metric ouptut for same dataset not supported."); - flowOutput.HttpOutput = httpOutput; + Ensure.EnsureNullElseThrowNotSupported(flowOutput.HttpOutput, "Multiple target httpost/metric output for same dataset not supported."); + flowOutput.HttpOutput = httpOutput; } break; } else { var eventhubOutput = ProcessOutputMetric(output); - Ensure.EnsureNullElseThrowNotSupported(flowOutput.EventHubOutput, "Multiple target eventHub/metric ouptut for same dataset not supported."); + Ensure.EnsureNullElseThrowNotSupported(flowOutput.EventHubOutput, "Multiple target eventHub/metric output for same dataset not supported."); flowOutput.EventHubOutput = eventhubOutput; break; } } case "blob": { - var blobOutput = await ProcessOutputBlob(configName, output); - Ensure.EnsureNullElseThrowNotSupported(flowOutput.BlobOutput, "Multiple target blob ouptut for same dataset not supported."); + var blobOutput = await ProcessOutputBlob(inputMode, configName, output); + Ensure.EnsureNullElseThrowNotSupported(flowOutput.BlobOutput, "Multiple target blob output for same dataset not supported."); flowOutput.BlobOutput = blobOutput; break; } case "local": { var blobOutput = ProcessOutputLocal(configName, output); - Ensure.EnsureNullElseThrowNotSupported(flowOutput.BlobOutput, "Multiple target blob ouptut for same dataset not supported."); + Ensure.EnsureNullElseThrowNotSupported(flowOutput.BlobOutput, "Multiple target blob output for same dataset not supported."); flowOutput.BlobOutput = blobOutput; break; } + case "sqlserver": + { + var sqlOutput = await ProcessOutputSql(configName, output); + Ensure.EnsureNullElseThrowNotSupported(flowOutput.SqlOutput, "Multiple target Sql output for same dataset not supported."); + flowOutput.SqlOutput = sqlOutput; + break; + } default: throw new NotSupportedException($"{output.Type} output type not supported"); } @@ -186,18 +194,61 @@ private async Task ProcessOutputs(FlowGuiOutput[] uiOutputs, R return fOutputList.ToArray(); } - private async Task ProcessOutputBlob(string configName, FlowGuiOutput uiOutput) + private string TransformPartitionFormat(string partitionFormat) + { + var parts = partitionFormat.Split(new char[] { ',', '/', ':', '-', ' ' }); + + string value = ""; + foreach (var part in parts) + { + value = ""; + switch (part.Substring(0, 1)) + { + case "s": + value = "%1$tS"; + break; + case "m": + value = "%1$tM"; + break; + case "h": + case "H": + value = "%1$tH"; + break; + case "d": + value = "%1$td"; + break; + case "M": + value = "%1$tm"; + break; + case "y": + value = "%1$ty"; + break; + default: + value = ""; + break; + } + + partitionFormat = partitionFormat.Replace(part, value); + + } + + return partitionFormat; + } + + private async Task ProcessOutputBlob(string inputMode, string configName, FlowGuiOutput uiOutput) { if (uiOutput != null && uiOutput.Properties != null) { var sparkKeyVaultName = Configuration[Constants.ConfigSettingName_RuntimeKeyVaultName]; string connectionString = await KeyVaultClient.ResolveSecretUriAsync(uiOutput.Properties.ConnectionString); - var accountName = ParseBlobAccountName(connectionString); - var blobPath = $"wasbs://{uiOutput.Properties.ContainerName}@{accountName}.blob.core.windows.net/{uiOutput.Properties.BlobPrefix}/%1$tY/%1$tm/%1$td/%1$tH/${{quarterBucket}}/${{minuteBucket}}"; + var accountName = ConfigHelper.ParseBlobAccountName(connectionString); + var timeFormat = inputMode != Constants.InputMode_Batching ? $"%1$tY/%1$tm/%1$td/%1$tH/${{quarterBucket}}/${{minuteBucket}}" : $"{TransformPartitionFormat(uiOutput.Properties.BlobPartitionFormat)}"; + var blobPath = $"wasbs://{uiOutput.Properties.ContainerName}@{accountName}.blob.core.windows.net/{uiOutput.Properties.BlobPrefix}/{timeFormat}"; var secretId = $"{configName}-output"; - var blobPathSecret = await KeyVaultClient.SaveSecretAsync(sparkKeyVaultName, secretId, blobPath, true); - await KeyVaultClient.SaveSecretAsync(sparkKeyVaultName, $"datax-sa-{accountName}", ParseBlobAccountKey(connectionString), false); + var sparkType = Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string value) ? value : null; + var blobPathSecret = await KeyVaultClient.SaveSecretAsync(sparkKeyVaultName, secretId, blobPath, sparkType, true); + await KeyVaultClient.SaveSecretAsync(sparkKeyVaultName, $"{Constants.AccountSecretPrefix}{accountName}", ConfigHelper.ParseBlobAccountKey(connectionString), sparkType, false); FlowBlobOutputSpec blobOutput = new FlowBlobOutputSpec() { @@ -246,7 +297,7 @@ private FlowEventHubOutputSpec ProcessOutputMetric(FlowGuiOutput uiOutput) FlowEventHubOutputSpec eventhubOutput = new FlowEventHubOutputSpec() { - ConnectionStringRef = KeyVaultUri.ComposeUri(sparkKeyVaultName, metricsEhConnectionStringKey), + ConnectionStringRef = KeyVaultUri.ComposeUri(sparkKeyVaultName, metricsEhConnectionStringKey, Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string sparkType) ? sparkType : null), CompressionType = "none", Format = "json" }; @@ -320,44 +371,72 @@ private FlowHttpOutputSpec ProcessLocalOutputMetric(string configName, string en } - /// - /// Parses the account name from connection string - /// - /// - /// account name - private string ParseBlobAccountName(string connectionString) + private async Task ProcessOutputSql(string configName, FlowGuiOutput uiOutput) + { + if (uiOutput != null && uiOutput.Properties != null) + { + var sparkKeyVaultName = Configuration[Constants.ConfigSettingName_RuntimeKeyVaultName]; + + string connectionString = await KeyVaultClient.ResolveSecretUriAsync(uiOutput.Properties.ConnectionString).ConfigureAwait(false); + + var database = GetValueFromJdbcConnection(connectionString, "database"); + var user = GetValueFromJdbcConnection(connectionString, "user"); + var pwd = GetValueFromJdbcConnection(connectionString, "password"); + var url = GetUrlFromJdbcConnection(connectionString); + + var sparkType = Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string value) ? value : null; + + // Save password and url in keyvault + var pwdSecretId = $"{configName}-outSqlPassword"; + var pwdRef = await KeyVaultClient.SaveSecretAsync(sparkKeyVaultName, pwdSecretId, pwd, sparkType, true).ConfigureAwait(false); + + var urlSecretId = $"{configName}-outSqlUrl"; + var urlRef = await KeyVaultClient.SaveSecretAsync(sparkKeyVaultName, urlSecretId, url, sparkType, true).ConfigureAwait(false); + + FlowSqlOutputSpec sqlOutput = new FlowSqlOutputSpec() + { + ConnectionStringRef = uiOutput.Properties.ConnectionString, + TableName = uiOutput.Properties.TableName, + WriteMode = uiOutput.Properties.WriteMode, + UseBulkInsert = uiOutput.Properties.UseBulkInsert, + DatabaseName = database, + User = user, + Password = pwdRef, + Url = urlRef + }; + return sqlOutput; + } + else + { + return null; + } + } + + private string GetValueFromJdbcConnection(string connectionString, string key) { - string matched; try { - matched = Regex.Match(connectionString, @"(?<=AccountName=)(.*)(?=;AccountKey)").Value; + Match match = Regex.Match(connectionString, $"{key}=([^;]*);", RegexOptions.IgnoreCase); + string value = match.Groups[1].Value; + return value; } catch (Exception) { - return "The connectionString does not have AccountName"; + throw new Exception($"{key} not found in jdbc connection string"); } - - return matched; } - - /// - /// Parses the account key from connection string - /// - /// - /// account key - private string ParseBlobAccountKey(string connectionString) + private string GetUrlFromJdbcConnection(string connectionString) { - string matched; try { - matched = Regex.Match(connectionString, @"(?<=AccountKey=)(.*)(?=;EndpointSuffix)").Value; + Match match = Regex.Match(connectionString, @"jdbc:sqlserver://(.*):", RegexOptions.IgnoreCase); + string value = match.Groups[1].Value; + return value; } catch (Exception) { - return "The connectionString does not have AccountKey"; + throw new Exception("url pattern not found in jdbc connecton string"); } - - return matched; } } } diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveReferenceData.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveReferenceData.cs index 06942cc4a..ce283ee48 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveReferenceData.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveReferenceData.cs @@ -52,6 +52,7 @@ public override async Task HandleSensitiveData(FlowGuiConfig guiC keyvaultName: RuntimeKeyVaultName.Value, secretName: secretName, secretValue: path, + sparkType: Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string sparkType) ? sparkType : null, hashSuffix: true); rd.Properties.Path = secretUri; diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveSparkJobParams.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveSparkJobParams.cs index 6cb10583f..1cb9cdb8b 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveSparkJobParams.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveSparkJobParams.cs @@ -9,7 +9,8 @@ using System.Composition; using System.Text; using System.Threading.Tasks; - +using DataX.Config.ConfigDataModel; + namespace DataX.Config.ConfigGeneration.Processor { /// @@ -21,11 +22,22 @@ public class ResolveSparkJobParams: ProcessorBase { public const string TokenName_SparkJobNumExecutors = "guiSparkJobNumExecutors"; public const string TokenName_SparkJobJobExecutorMemory = "guiSparkJobExecutorMemory"; - + public const string TokenName_SparkJobDatabricksMinWorkers = "guiSparkJobDatabricksMinWorkers"; + public const string TokenName_SparkJobDatabricksMaxWorkers = "guiSparkJobDatabricksMaxWorkers"; + public const string TokenName_DatabricksToken = "guiSparkDatabricksToken"; + public const string TokenName_SparkJobDatabricksAutoScale = "guiSparkJobDatabricksAutoScale"; + + [ImportingConstructor] + public ResolveSparkJobParams(ConfigGenConfiguration conf) + { + Configuration = conf; + } + private ConfigGenConfiguration Configuration { get; } public override async Task Process(FlowDeploymentSession flowToDeploy) { var guiConfig = flowToDeploy.Config?.GetGuiConfig(); if (guiConfig == null) + { // If guiConfig is empty, get the number of executors from job common token and convert it to integer var executorsString = flowToDeploy.Config?.CommonProcessor?.JobCommonTokens?.GetOrDefault("sparkJobNumExecutors", null); @@ -46,7 +58,7 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) { throw new ConfigGenerationException($"Invalid value for process.jobconfig.jobNumExecutors:'{numExecutorsString}'."); } - + flowToDeploy.SetObjectToken(TokenName_SparkJobNumExecutors, numExecutors); // Setting TokenName_SparkJobJobExecutorMemory @@ -57,7 +69,35 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) } flowToDeploy.SetStringToken(TokenName_SparkJobJobExecutorMemory, $"{jobExecutorMemory}m"); - + var sparkType = Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string value) ? value : null; + if (sparkType == Config.ConfigDataModel.Constants.SparkTypeDataBricks) + { + // Setting TokenName_SparkJobDatabricksMinWorkers + var jobDatabricksMinWorkersString = guiConfig?.Process?.JobConfig?.JobDatabricksMinWorkers; + if (!int.TryParse(jobDatabricksMinWorkersString, out int jobDatabricksMinWorkers)) + { + throw new ConfigGenerationException($"Invalid value for process.jobconfig.jobDatabricksMinWorkers:'{jobDatabricksMinWorkersString}'."); + } + + flowToDeploy.SetStringToken(TokenName_SparkJobDatabricksMinWorkers, $"{jobDatabricksMinWorkers}"); + + // Setting TokenName_SparkJobDatabricksMaxWorkers + var jobDatabricksMaxWorkersString = guiConfig?.Process?.JobConfig?.JobDatabricksMaxWorkers; + if (!int.TryParse(jobDatabricksMaxWorkersString, out int jobDatabricksMaxWorkers)) + { + throw new ConfigGenerationException($"Invalid value for process.jobconfig.jobDatabricksMaxWorkers:'{jobDatabricksMaxWorkersString}'."); + } + + flowToDeploy.SetStringToken(TokenName_SparkJobDatabricksMaxWorkers, $"{jobDatabricksMaxWorkers}"); + + // Setting TokenName_DatabricksToken + var jobDatabricksTokenString = guiConfig?.DatabricksToken; + flowToDeploy.SetStringToken(TokenName_DatabricksToken, $"{jobDatabricksTokenString}"); + + // Setting TokenName_SparkJobDatabricksAutoScale + var jobDatabricksAutoScaleString = guiConfig?.Process?.JobConfig?.JobDatabricksAutoScale; + flowToDeploy.SetStringToken(TokenName_SparkJobDatabricksAutoScale, $"{jobDatabricksAutoScaleString}"); + } await Task.Yield(); return "done"; } diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveStateTable.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveStateTable.cs index 3f2488a21..c79e1fe66 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveStateTable.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveStateTable.cs @@ -12,6 +12,7 @@ using System.Linq; using System.Text; using System.Threading.Tasks; +using DataX.Config.ConfigDataModel; namespace DataX.Config.ConfigGeneration.Processor { @@ -23,7 +24,14 @@ namespace DataX.Config.ConfigGeneration.Processor public class ResolveStateTable: ProcessorBase { public const string TokenName_StateTables = "processStateTables"; - + + [ImportingConstructor] + public ResolveStateTable(ConfigGenConfiguration conf) + { + Configuration = conf; + } + private ConfigGenConfiguration Configuration { get; } + public override async Task Process(FlowDeploymentSession flowToDeploy) { var config = flowToDeploy.Config; @@ -33,7 +41,7 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) return "no gui input, skipped."; } - var rulesCode = flowToDeploy.GetAttachment(GenerateTransformFile.AttachmentName_CodeGenObject); + var rulesCode = flowToDeploy.GetAttachment(PrepareTransformFile.AttachmentName_CodeGenObject); Ensure.NotNull(rulesCode, "rulesCode"); Ensure.NotNull(rulesCode.MetricsRoot, "rulesCode.MetricsRoot"); Ensure.NotNull(rulesCode.MetricsRoot.metrics, "rulesCode.MetricsRoot.metrics"); @@ -58,7 +66,9 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) private string ConstructStateTablePath(string flowName, string tableName) { - return $"hdfs://mycluster/datax/{flowName}/{tableName}/"; + var sparkType = Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string value) ? value : null; + var prefix = (sparkType == Constants.SparkTypeDataBricks) ? Constants.PrefixDbfs : Constants.PrefixHdfs; + return $"{prefix}mycluster/datax/{flowName}/{tableName}/"; } } } diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveStreamming.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveStreamming.cs index a6dfcad1f..d6bc277ff 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveStreamming.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveStreamming.cs @@ -2,6 +2,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using DataX.Config.ConfigDataModel; using Newtonsoft.Json; using System; using System.Collections.Generic; @@ -20,13 +21,21 @@ public class ResolveStreaming : ProcessorBase { public const string TokenName_InputStreamingCheckpointDir = "inputStreamingCheckpointDir"; public const string TokenName_InputStreamingInterval = "inputStreamingIntervalInSeconds"; - + + [ImportingConstructor] + public ResolveStreaming(ConfigGenConfiguration conf) + { + Configuration = conf; + } + private ConfigGenConfiguration Configuration { get; } + public override async Task Process(FlowDeploymentSession flowToDeploy) { var config = flowToDeploy.Config; var guiConfig = config.GetGuiConfig(); - - flowToDeploy.SetStringToken(TokenName_InputStreamingCheckpointDir, $"hdfs://mycluster/dataxdirect/{JobMetadata.TokenPlaceHolder_JobName}/streaming/checkpoints"); + var sparkType = Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string value) ? value : null; + var inputStreamingCheckpointDirPrefix = (sparkType == Constants.SparkTypeDataBricks) ? Constants.PrefixDbfs : Constants.PrefixHdfs; + flowToDeploy.SetStringToken(TokenName_InputStreamingCheckpointDir, $"{inputStreamingCheckpointDirPrefix}mycluster/dataxdirect/{JobMetadata.TokenPlaceHolder_JobName}/streaming/checkpoints"); var intervalInSeconds = guiConfig?.Input?.Properties?.WindowDuration; flowToDeploy.SetStringToken(TokenName_InputStreamingInterval, intervalInSeconds); diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveTimeWindow.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveTimeWindow.cs index f9a62e46f..85c82cbac 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveTimeWindow.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_ResolveTimeWindow.cs @@ -32,7 +32,7 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) { return "no gui input, skipped."; } - var rulesCode = flowToDeploy.GetAttachment(GenerateTransformFile.AttachmentName_CodeGenObject); + var rulesCode = flowToDeploy.GetAttachment(PrepareTransformFile.AttachmentName_CodeGenObject); Ensure.NotNull(rulesCode, "rulesCode"); Ensure.NotNull(rulesCode.MetricsRoot, "rulesCode.MetricsRoot"); Ensure.NotNull(rulesCode.MetricsRoot.metrics, "rulesCode.MetricsRoot.metrics"); diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S550_ResolveBatchInput.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S550_ResolveBatchInput.cs new file mode 100644 index 000000000..00f16081c --- /dev/null +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S550_ResolveBatchInput.cs @@ -0,0 +1,142 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Config.ConfigDataModel; +using DataX.Config.ConfigDataModel.RuntimeConfig; +using DataX.Config.Utility; +using DataX.Contract; +using System; +using System.Composition; +using System.Globalization; +using System.Linq; +using System.Text.RegularExpressions; +using System.Threading.Tasks; + +namespace DataX.Config.ConfigGeneration.Processor +{ + /// + /// Produce the time window section + /// + [Shared] + [Export(typeof(IFlowDeploymentProcessor))] + public class ResolveBatchInput : ProcessorBase + { + public const string TokenName_InputBatching = "inputBatching"; + + [ImportingConstructor] + public ResolveBatchInput(ConfigGenConfiguration configuration, IKeyVaultClient keyVaultClient) + { + Configuration = configuration; + KeyVaultClient = keyVaultClient; + } + + private ConfigGenConfiguration Configuration { get; } + private IKeyVaultClient KeyVaultClient { get; } + + public override async Task HandleSensitiveData(FlowGuiConfig guiConfig) + { + if (guiConfig?.Input?.Mode == Constants.InputMode_Batching) + { + var runtimeKeyVaultName = Configuration[Constants.ConfigSettingName_RuntimeKeyVaultName]; + Ensure.NotNull(runtimeKeyVaultName, "runtimeKeyVaultName"); + + var sparkType = Configuration.TryGet(Constants.ConfigSettingName_SparkType, out string value) ? value : null; + + for (int i = 0; i < guiConfig?.Input?.Batch?.Length; i++) + { + // Replace Input Path + var input = guiConfig?.Input?.Batch[i]; + var inputConnection = input.Properties.Connection; + if (!string.IsNullOrEmpty(inputConnection) && !KeyVaultUri.IsSecretUri(inputConnection)) + { + var secretName = $"{guiConfig.Name}-input-{i}-inputConnection"; + var secretId = await KeyVaultClient.SaveSecretAsync(runtimeKeyVaultName, secretName, inputConnection, sparkType).ConfigureAwait(false); + input.Properties.Connection = secretId; + + var accountName = ConfigHelper.ParseBlobAccountName(inputConnection); + await KeyVaultClient.SaveSecretAsync(runtimeKeyVaultName, $"{Constants.AccountSecretPrefix}{accountName}", ConfigHelper.ParseBlobAccountKey(inputConnection), sparkType, false); + } + + var inputPath = input.Properties.Path; + if (!string.IsNullOrEmpty(inputPath) && !KeyVaultUri.IsSecretUri(inputPath)) + { + var secretName = $"{guiConfig.Name}-input-{i}-inputPath"; + var secretId = await KeyVaultClient.SaveSecretAsync(runtimeKeyVaultName, secretName, inputPath, Configuration[Constants.ConfigSettingName_SparkType]).ConfigureAwait(false); + input.Properties.Path = secretId; + } + } + } + + return guiConfig; + } + + public override async Task Process(FlowDeploymentSession flowToDeploy) + { + var inputConfig = flowToDeploy.Config.GetGuiConfig(); + if (inputConfig == null) + { + return "no gui input, skipped"; + } + + if (inputConfig?.Input?.Mode == Constants.InputMode_Batching) + { + var inputBatching = inputConfig?.Input?.Batch ?? Array.Empty(); + var specsTasks = inputBatching.Select(async rd => + { + var connectionString = await KeyVaultClient.ResolveSecretUriAsync(rd.Properties.Connection).ConfigureAwait(false); + var inputPath = await KeyVaultClient.ResolveSecretUriAsync(rd.Properties.Path).ConfigureAwait(false); + + return new InputBatchingSpec() + { + Name = ConfigHelper.ParseBlobAccountName(connectionString), + Path = rd.Properties.Path, + Format = rd.Properties.FormatType, + CompressionType = rd.Properties.CompressionType, + ProcessStartTime = "", + ProcessEndTime = "", + PartitionIncrement = GetPartitionIncrement(inputPath).ToString(CultureInfo.InvariantCulture), + }; + }).ToArray(); + + var specs = await Task.WhenAll(specsTasks).ConfigureAwait(false); + + flowToDeploy.SetAttachment(TokenName_InputBatching, specs); + } + + return "done"; + } + + private static long GetPartitionIncrement(string path) + { + Regex regex = new Regex(@"\{([yMdHhmsS\-\/.,: ]+)\}*", RegexOptions.IgnoreCase); + Match mc = regex.Match(path); + + if (mc != null && mc.Success && mc.Groups.Count > 1) + { + var value = mc.Groups[1].Value.Trim(); + + value = value.Replace(@"[\/:\s-]", "", StringComparison.InvariantCultureIgnoreCase).Replace(@"(.)(?=.*\1)", "", StringComparison.InvariantCultureIgnoreCase); + + if (value.Contains("h", StringComparison.InvariantCultureIgnoreCase)) + { + return 1 * 60; + } + else if (value.Contains("d", StringComparison.InvariantCultureIgnoreCase)) + { + return 1 * 60 * 24; + } + else if (value.Contains("M", StringComparison.InvariantCulture)) + { + return 1 * 60 * 24 * 30; + } + else if (value.Contains("y", StringComparison.InvariantCultureIgnoreCase)) + { + return 1 * 60 * 24 * 30 * 12; + } + } + + return 1; + } + } +} diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S600_GenerateJobConfig.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S600_GenerateJobConfig.cs index c209defc1..b4d163b6e 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S600_GenerateJobConfig.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S600_GenerateJobConfig.cs @@ -3,15 +3,9 @@ // Licensed under the MIT License // ********************************************************************* using DataX.Config.ConfigDataModel; -using DataX.Config.ConfigGeneration.Processor; -using DataX.Config.Templating; using DataX.Config.Utility; using DataX.Contract; -using System; -using System.Collections.Generic; using System.Composition; -using System.Linq; -using System.Text; using System.Threading.Tasks; namespace DataX.Config.ConfigGeneration.Processor @@ -25,7 +19,6 @@ public class GenerateJobConfig : ProcessorBase { public const string ParameterObjectName_DefaultJobConfig = "defaultJobConfig"; public const string TokenName_JobConfigContent = "jobConfigContent"; - public const string TokenName_JobConfigFilePath = "jobConfigFilePath"; [ImportingConstructor] public GenerateJobConfig(JobDataManager jobs) @@ -44,10 +37,14 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) { var flowConfig = flowToDeploy.Config; + if (flowConfig.GetGuiConfig()?.Input?.Mode == Constants.InputMode_Batching) + { + return "done"; + } + // set the default job config var defaultJobConfig = JsonConfig.From(flowConfig.CommonProcessor?.Template); Ensure.NotNull(defaultJobConfig, "defaultJobConfig"); - flowToDeploy.SetAttachment(ParameterObjectName_DefaultJobConfig, defaultJobConfig); // Deploy job configs var jobsToDeploy = flowToDeploy?.GetJobs(); @@ -78,10 +75,15 @@ private void GenerateJobConfigContent(JobDeploymentSession job, string destFolde Ensure.NotNull(newJobConfig, "newJobConfig"); job.SetJsonToken(TokenName_JobConfigContent, newJobConfig); - var destinationPath = ResourcePathUtil.Combine(destFolder, job.Name + ".json"); - job.SetStringToken(TokenName_JobConfigFilePath, destinationPath); - } + var jc = new JobConfig + { + Content = newJobConfig.ToString(), + FilePath = ResourcePathUtil.Combine(destFolder, job.Name + ".conf"), + SparkJobName = job.SparkJobName, + }; + job.JobConfigs.Add(jc); + } /// /// Delete the runtime job configs /// diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S600_GenerateJobConfigBatch.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S600_GenerateJobConfigBatch.cs new file mode 100644 index 000000000..4d9838403 --- /dev/null +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S600_GenerateJobConfigBatch.cs @@ -0,0 +1,506 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Config.ConfigDataModel; +using DataX.Config.ConfigDataModel.RuntimeConfig; +using DataX.Config.Utility; +using DataX.Contract; +using Newtonsoft.Json.Linq; +using System; +using System.Collections.Generic; +using System.Composition; +using System.Globalization; +using System.Text.RegularExpressions; +using System.Threading.Tasks; + +namespace DataX.Config.ConfigGeneration.Processor +{ + /// + /// Produce job config content and file path + /// + [Shared] + [Export(typeof(IFlowDeploymentProcessor))] + public class GenerateJobConfigBatch : ProcessorBase + { + public const string TokenName__DefaultJobConfig = "defaultJobConfig"; + public const string TokenName_InputBatching = "inputBatching"; + + [ImportingConstructor] + public GenerateJobConfigBatch(JobDataManager jobs, FlowDataManager flowData, IKeyVaultClient keyVaultClient) + { + this.JobData = jobs; + this.FlowData = flowData; + this.KeyVaultClient = keyVaultClient; + } + + private JobDataManager JobData { get; } + private FlowDataManager FlowData { get; } + private IKeyVaultClient KeyVaultClient { get; } + + public override int GetOrder() + { + return 600; + } + + public override async Task Process(FlowDeploymentSession flowToDeploy) + { + var flowConfig = flowToDeploy.Config; + + if (flowConfig.GetGuiConfig()?.Input?.Mode != Constants.InputMode_Batching) + { + return "done"; + } + + // set the default job config + var defaultJobConfig = JsonConfig.From(flowConfig.CommonProcessor?.Template); + Ensure.NotNull(defaultJobConfig, "defaultJobConfig"); + flowToDeploy.SetAttachment(TokenName__DefaultJobConfig, defaultJobConfig); + + // Deploy job configs + var jobsToDeploy = flowToDeploy?.GetJobs(); + if (jobsToDeploy != null) + { + foreach (var job in jobsToDeploy) + { + await GenerateJobConfigContent(job, job.GetTokenString(PrepareJobConfigVariables.TokenName_RuntimeConfigFolder), defaultJobConfig).ConfigureAwait(false); + } + + return "done"; + } + else + { + await Task.Yield(); + return "no jobs, skipped"; + } + } + + /// + /// Generate job config with the jobs scheduled + /// + /// + private async Task GenerateJobConfigContent(JobDeploymentSession job, string destFolder, JsonConfig defaultJobConfig) + { + Ensure.NotNull(destFolder, "destFolder"); + Ensure.NotNull(defaultJobConfig, "defaultJobConfig"); + + var inputConfig = job.Flow.Config?.GetGuiConfig(); + if (inputConfig != null) + { + var jcons = new List(); + + for (int i = 0; i < inputConfig.BatchList.Length; i++) + { + job.JobConfigs.AddRange(await GetJobConfig(job, inputConfig.BatchList[i], destFolder, defaultJobConfig, i).ConfigureAwait(false)); + } + } + } + + /// + /// Prepare for scheduling batch jobs + /// e.g. Calculate the processTime and startTime and endTime + /// + /// + private async Task> GetJobConfig(JobDeploymentSession job, FlowGuiInputBatchJob batchingJob, + string destFolder, + JsonConfig defaultJobConfig, + int index) + { + bool isOneTime = batchingJob.Type == Constants.Batch_OneTime; + var jQueue = new List(); + var batchProps = batchingJob.Properties; + + if (batchingJob.Disabled || batchProps.StartTime == null || (isOneTime && batchProps.EndTime == null)) + { + return jQueue; + } + + var configStartTime = (DateTime)batchProps.StartTime; + var configEndTime = batchProps.EndTime; + + var interval = TranslateInterval(batchProps.Interval, batchProps.IntervalType); + var delay = TranslateDelay(batchProps.Delay, batchProps.DelayType); + var window = TranslateWindow(batchProps.Window, batchProps.WindowType); + + var currentTime = DateTime.UtcNow; + + if (!isOneTime) + { + if (currentTime < configStartTime || (configEndTime != null && configEndTime < currentTime)) + { + await DisableBatchConfig(job.Flow.Config, index).ConfigureAwait(false); + return jQueue; + } + } + + DateTime startTime; + DateTime? endTime; + + var prefix = ""; + + if (!isOneTime) + { + if (string.IsNullOrEmpty(batchProps.LastProcessedTime)) + { + startTime = configStartTime; + endTime = currentTime; + } + else + { + var lastProcessedTimeFromConfig = UnixTimestampToDateTime(batchProps.LastProcessedTime); + var startTimeBasedOnLastProcessedTImeFromConfig = lastProcessedTimeFromConfig.Add(interval); + + startTime = startTimeBasedOnLastProcessedTImeFromConfig; + endTime = currentTime; + } + } + else + { + prefix = "-OneTime"; + + startTime = configStartTime; + endTime = configEndTime; + } + + DateTime lastProcessingTime = new DateTime(); + for (var processingTime = startTime; processingTime <= endTime; processingTime += interval) + { + lastProcessingTime = processingTime; + var processingTimeBasedOnInterval = NormalizeTimeBasedOnInterval(processingTime, batchProps.IntervalType, new TimeSpan()); + var processingTimeBasedOnDelay = NormalizeTimeBasedOnInterval(processingTime, batchProps.IntervalType, delay); + JobConfig jc = ScheduleSingleJob(job, destFolder, defaultJobConfig, isOneTime, interval, window, processingTimeBasedOnDelay, processingTimeBasedOnInterval, prefix); + jQueue.Add(jc); + } + + if (!isOneTime) + { + if (lastProcessingTime != DateTime.MinValue) + { + var uTimestamp = DateTimeToUnixTimestamp(lastProcessingTime); + if (uTimestamp > 0) + { + var ret = await UpdateLastProcessedTime(job.Flow.Config, index, uTimestamp).ConfigureAwait(false); + if (!ret.IsSuccess) + { + throw new ConfigGenerationException(ret.Message); + } + } + } + } + else + { + // OneTime + var ret = await DisableBatchConfig(job.Flow.Config, index).ConfigureAwait(false); + if (!ret.IsSuccess) + { + throw new ConfigGenerationException(ret.Message); + } + + } + + return jQueue; + } + + /// + /// Schedule a batch job + /// + /// + private JobConfig ScheduleSingleJob(JobDeploymentSession job, string destFolder, JsonConfig defaultJobConfig, bool isOneTime, TimeSpan interval, TimeSpan window, DateTime processTime, DateTime scheduledTime, string prefix = "") + { + var ps_s = processTime; + var ps_e = processTime.Add(interval).AddMilliseconds(-1); //ENDTIME + + var pe_s = ps_s.Add(-window); + var pe_e = ps_e.Add(-window); // STARTTIME + + var dateString = ConvertDateToString(scheduledTime); + var suffix = prefix + $"-{Regex.Replace(dateString, "[^0-9]", "")}"; + var jobName = job.Name + suffix; + job.SetStringToken("name", jobName); + + Ensure.NotNull(defaultJobConfig, "defaultJobConfig"); + + var processStartTime = ConvertDateToString(pe_e); + var processEndTime = ConvertDateToString(ps_e); + + destFolder = GetJobConfigFilePath(isOneTime, dateString, destFolder); + var jc = new JobConfig + { + Content = GetBatchConfigContent(job, defaultJobConfig.ToString(), processStartTime, processEndTime), + FilePath = ResourcePathUtil.Combine(destFolder, job.Name + ".conf"), + Name = jobName, + SparkJobName = job.SparkJobName + suffix, + ProcessStartTime = processStartTime, + ProcessEndTime = processEndTime, + ProcessingTime = dateString, + IsOneTime = isOneTime + }; + + return jc; + } + + /// + /// Get the job config path based on the job type + /// + /// + private static string GetJobConfigFilePath(bool isOneTime, string partitionName, string baseFolder) + { + var oneTimeFolderName = ""; + if (isOneTime) + { + oneTimeFolderName = $"OneTime/{Regex.Replace(partitionName, "[^0-9]", "")}"; + } + else + { + oneTimeFolderName = $"Recurring/{Regex.Replace(partitionName, "[^0-9]", "")}"; + } + + return ResourcePathUtil.Combine(baseFolder, oneTimeFolderName); + } + + /// + /// Get a batch job config + /// + /// + private static string GetBatchConfigContent(JobDeploymentSession job, string content, string processStartTime, string processEndTime) + { + var specsBackup = job.GetAttachment(TokenName_InputBatching); + + foreach (var spec in specsBackup) + { + spec.ProcessStartTime = processStartTime; + spec.ProcessEndTime = processEndTime; + } + + job.SetObjectToken(TokenName_InputBatching, specsBackup); + + var jsonContent = job.Tokens.Resolve(content); + + return jsonContent; + } + + public override async Task Delete(FlowDeploymentSession flowToDelete) + { + var flowConfig = flowToDelete.Config; + var runtimeConfigsFolder = flowConfig.GetJobConfigDestinationFolder(); + + flowToDelete.SetStringToken(PrepareJobConfigVariables.TokenName_RuntimeConfigFolder, runtimeConfigsFolder); + var folderToDelete = flowToDelete.GetTokenString(PrepareJobConfigVariables.TokenName_RuntimeConfigFolder); + return await this.JobData.DeleteConfigs(folderToDelete); + } + + /// + /// Normalize the datetime base on interval and delay + /// + /// + private static DateTime NormalizeTimeBasedOnInterval(DateTime dateTime, string intervalType, TimeSpan delay) + { + dateTime = dateTime.Add(-delay); + int second = dateTime.Second; + int minute = dateTime.Minute; + int hour = dateTime.Hour; + int day = dateTime.Day; + int month = dateTime.Month; + int year = dateTime.Year; + + switch (intervalType) + { + case "min": + { + second = 0; + break; + } + case "hour": + { + second = 0; + minute = 0; + break; + } + default: + { + second = 0; + minute = 0; + hour = 0; + break; + } + } + + return new DateTime(year, month, day, hour, minute, second); + } + + /// + /// Translate Interval to a timespan value + /// + /// + private static int TranslateIntervalHelper(string unit) + { + switch (unit) + { + case "min": + return 1; + case "hour": + return 60; + default: + return 60 * 24; + } + } + + /// + /// Generate a timespan for Interval + /// + /// + private static TimeSpan TranslateInterval(string value, string unit) + { + var translatedValue = TranslateIntervalHelper(unit); + int multiplier = Convert.ToInt32(value, CultureInfo.InvariantCulture); + translatedValue = translatedValue * multiplier; + + var timeSpan = new TimeSpan(0, 0, translatedValue, 0, 0); + return timeSpan; + } + + /// + /// Generate a timespan for Window + /// + /// + private static TimeSpan TranslateWindow(string value, string unit) + { + var translatedValue = TranslateIntervalHelper(unit); + int multiplier = Convert.ToInt32(value, CultureInfo.InvariantCulture); + translatedValue = translatedValue * multiplier; + + var timeSpan = new TimeSpan(0, 0, translatedValue - 1, 59, 59); + return timeSpan; + } + + /// + /// Generate a timespan for Delay + /// + /// + private static TimeSpan TranslateDelay(string value, string unit) + { + var translatedValue = TranslateIntervalHelper(unit); + int multiplier = Convert.ToInt32(value, CultureInfo.InvariantCulture); + translatedValue = translatedValue * multiplier; + + var timeSpan = new TimeSpan(0, 0, translatedValue, 0, 0); + return timeSpan; + } + + /// + /// Convert a datetime to string in an expected format + /// Implement this specific helper function since using "o" doesn't work + /// + /// + private static string ConvertDateToString(DateTime dateTime) + { + return dateTime.ToString("s", CultureInfo.InvariantCulture) + "Z"; + } + + /// + /// Convert a datetime to an epoch timestamp + /// + /// + public static long DateTimeToUnixTimestamp(DateTime dateTime) + { + DateTime unixStart = new DateTime(1970, 1, 1, 0, 0, 0, 0, System.DateTimeKind.Utc); + TimeSpan elapsedTime = dateTime - unixStart; + return (long)elapsedTime.TotalSeconds; + } + + /// + /// Convert an epoch timestamp to a datetime + /// + /// + public static DateTime UnixTimestampToDateTime(string unixTime) + { + double uTime = Convert.ToDouble(unixTime, CultureInfo.InvariantCulture); + DateTime unixStart = new DateTime(1970, 1, 1, 0, 0, 0, 0, System.DateTimeKind.Utc); + return unixStart.AddSeconds(uTime); + } + + /// + /// Disable a batch job in the config + /// + /// + private async Task DisableBatchConfig(FlowConfig config, int index) + { + var existingFlow = await FlowData.GetByName(config.Name).ConfigureAwait(false); + Result result = null; + if (existingFlow != null) + { + var gui = config.GetGuiConfig(); + if (gui != null) + { + var batch = gui.BatchList[index]; + batch.Disabled = true; + + config.Gui = JObject.FromObject(gui); + result = await FlowData.UpdateGuiForFlow(config.Name, config.Gui).ConfigureAwait(false); + } + } + + return result; + } + + /// + /// Update the last processed time for a batch job in the config + /// + /// + private async Task UpdateLastProcessedTime(FlowConfig config, int index, long value) + { + var existingFlow = await FlowData.GetByName(config.Name).ConfigureAwait(false); + Result result = null; + if (existingFlow != null) + { + var gui = config.GetGuiConfig(); + if (gui != null) + { + var batch = gui.BatchList[index]; + batch.Properties.LastProcessedTime = value.ToString(CultureInfo.InvariantCulture); + + config.Gui = JObject.FromObject(gui); + result = await FlowData.UpdateGuiForFlow(config.Name, config.Gui).ConfigureAwait(false); + } + } + + return result; + } + + /// + /// Get a partition increment value using the given blob path + /// + /// + private static long GetPartitionIncrement(string path) + { + Regex regex = new Regex(@"\{([yMdHhmsS\-\/.,: ]+)\}*", RegexOptions.IgnoreCase); + Match mc = regex.Match(path); + + if (mc != null && mc.Success && mc.Groups.Count > 1) + { + var value = mc.Groups[1].Value.Trim(); + + value = value.Replace(@"[\/:\s-]", "", StringComparison.InvariantCultureIgnoreCase).Replace(@"(.)(?=.*\1)", "", StringComparison.InvariantCultureIgnoreCase); + + if (value.Contains("h", StringComparison.InvariantCultureIgnoreCase)) + { + return 1 * 60; + } + else if (value.Contains("d", StringComparison.InvariantCultureIgnoreCase)) + { + return 1 * 60 * 24; + } + else if (value.Contains("M", StringComparison.InvariantCulture)) + { + return 1 * 60 * 24 * 30; + } + else if (value.Contains("y", StringComparison.InvariantCultureIgnoreCase)) + { + return 1 * 60 * 24 * 30 * 12; + } + } + + return 1; + } + } +} diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S610_GenerateTransformFile.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S610_GenerateTransformFile.cs new file mode 100644 index 000000000..c291572d9 --- /dev/null +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S610_GenerateTransformFile.cs @@ -0,0 +1,69 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Newtonsoft.Json; +using DataX.Config.Utility; +using DataX.Contract; +using DataX.Flow.CodegenRules; +using System.Composition; +using System.Linq; +using System.Threading.Tasks; +using DataX.Config.ConfigDataModel; + +namespace DataX.Config.ConfigGeneration.Processor +{ + /// + /// Produce the transform/query file section + /// + [Shared] + [Export(typeof(IFlowDeploymentProcessor))] + public class GenerateTransformFile : ProcessorBase + { + private ConfigGenConfiguration Configuration { get; } + private IKeyVaultClient KeyVaultClient { get; } + private IRuntimeConfigStorage RuntimeStorage { get; } + + [ImportingConstructor] + public GenerateTransformFile(IKeyVaultClient keyvaultClient, IRuntimeConfigStorage runtimeStorage, ConfigGenConfiguration conf) + { + KeyVaultClient = keyvaultClient; + RuntimeStorage = runtimeStorage; + Configuration = conf; + } + + public override int GetOrder() + { + return 610; + } + + public override async Task Process(FlowDeploymentSession flowToDeploy) + { + var jobs = flowToDeploy.GetJobs(); + if (jobs == null || !jobs.Where(j => j.JobConfigs.Any()).Any()) + { + return "no jobs, skipped"; + } + + var config = flowToDeploy.Config; + var guiConfig = config?.GetGuiConfig(); + if (guiConfig == null) + { + return "no gui input, skipped."; + } + + var rulesCode = flowToDeploy.GetAttachment(PrepareTransformFile.AttachmentName_CodeGenObject); + Ensure.NotNull(rulesCode, "rulesCode"); + + var runtimeConfigBaseFolder = flowToDeploy.GetTokenString(PrepareJobConfigVariables.TokenName_RuntimeConfigFolder); + Ensure.NotNull(runtimeConfigBaseFolder, "runtimeConfigBaseFolder"); + + var filePath = ResourcePathUtil.Combine(runtimeConfigBaseFolder, $"{config.Name}-combined.txt"); + var transformFilePath = await RuntimeStorage.SaveFile(filePath, rulesCode.Code); + var transformFileSecret = flowToDeploy.GetTokenString(PrepareTransformFile.TokenName_TransformFile); + await KeyVaultClient.SaveSecretAsync(transformFileSecret, transformFilePath); + + return "done"; + } + } +} diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_GenerateProjectionFile.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S630_GenerateProjectionFile.cs similarity index 70% rename from Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_GenerateProjectionFile.cs rename to Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S630_GenerateProjectionFile.cs index d03927ad5..dff012a6f 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_GenerateProjectionFile.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S630_GenerateProjectionFile.cs @@ -5,9 +5,11 @@ using DataX.Config.ConfigDataModel; using DataX.Config.Utility; using DataX.Contract; +using Newtonsoft.Json.Linq; using System; using System.Collections.Generic; using System.Composition; +using System.Linq; using System.Text; using System.Threading.Tasks; @@ -18,22 +20,33 @@ namespace DataX.Config.ConfigGeneration.Processor /// [Shared] [Export(typeof(IFlowDeploymentProcessor))] - public class GenerateProjectionFile: ProcessorBase + public class GenerateProjectionFile : ProcessorBase { - public const string TokenName_ProjectionFiles = "processProjections"; - [ImportingConstructor] - public GenerateProjectionFile(IRuntimeConfigStorage runtimeStorage, IKeyVaultClient keyvaultClient) + public GenerateProjectionFile(IRuntimeConfigStorage runtimeStorage, IKeyVaultClient keyvaultClient, ConfigGenConfiguration conf) { RuntimeStorage = runtimeStorage; KeyVaultClient = keyvaultClient; + Configuration = conf; } + public override int GetOrder() + { + return 630; + } + + private ConfigGenConfiguration Configuration { get; } private IRuntimeConfigStorage RuntimeStorage { get; } private IKeyVaultClient KeyVaultClient { get; } public override async Task Process(FlowDeploymentSession flowToDeploy) { + var jobs = flowToDeploy.GetJobs(); + if (jobs == null || !jobs.Where(j => j.JobConfigs.Any()).Any()) + { + return "no jobs, skipped"; + } + var config = flowToDeploy.Config; var guiConfig = config?.GetGuiConfig(); if (guiConfig == null) @@ -53,9 +66,12 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) var filePath = ResourcePathUtil.Combine(runtimeConfigBaseFolder, "projection.txt"); var savedFile = await RuntimeStorage.SaveFile(filePath, finalProjections); - var secretName = $"{config.Name}-projectionfile"; - var savedSecretId = await KeyVaultClient.SaveSecretAsync(runtimeKeyVaultName, secretName, savedFile); - flowToDeploy.SetObjectToken(TokenName_ProjectionFiles, new string[] {savedSecretId}); + var tokenValue = flowToDeploy.GetTokenString(PrepareProjectionFile.TokenName_ProjectionFiles); + var projectionFileSecret = JArray.Parse(tokenValue).FirstOrDefault()?.Value(); + if (!string.IsNullOrEmpty(projectionFileSecret)) + { + await KeyVaultClient.SaveSecretAsync(projectionFileSecret, savedFile); + } return "done"; } diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_GenerateSchemaFile.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S630_GenerateSchemaFile.cs similarity index 75% rename from Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_GenerateSchemaFile.cs rename to Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S630_GenerateSchemaFile.cs index de681999e..d6328ea47 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S500_GenerateSchemaFile.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S630_GenerateSchemaFile.cs @@ -9,6 +9,7 @@ using System.Collections.Generic; using System.Composition; using System.IO; +using System.Linq; using System.Text; using System.Threading.Tasks; @@ -19,22 +20,33 @@ namespace DataX.Config.ConfigGeneration.Processor /// [Shared] [Export(typeof(IFlowDeploymentProcessor))] - public class GenerateSchemaFile: ProcessorBase + public class GenerateSchemaFile : ProcessorBase { - public const string TokenName_InputSchemaFilePath = "inputSchemaFilePath"; - [ImportingConstructor] - public GenerateSchemaFile(IKeyVaultClient keyvaultClient, IRuntimeConfigStorage runtimeStorage) + public GenerateSchemaFile(IKeyVaultClient keyvaultClient, IRuntimeConfigStorage runtimeStorage, ConfigGenConfiguration conf) { KeyVaultClient = keyvaultClient; RuntimeStorage = runtimeStorage; + Configuration = conf; + } + + public override int GetOrder() + { + return 630; } + private ConfigGenConfiguration Configuration { get; } private IKeyVaultClient KeyVaultClient { get; } private IRuntimeConfigStorage RuntimeStorage { get; } public override async Task Process(FlowDeploymentSession flowToDeploy) { + var jobs = flowToDeploy.GetJobs(); + if (jobs == null || !jobs.Where(j => j.JobConfigs.Any()).Any()) + { + return "no jobs, skipped"; + } + var config = flowToDeploy.Config; var guiConfig = config?.GetGuiConfig(); if (guiConfig == null) @@ -54,8 +66,8 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) var filePath = ResourcePathUtil.Combine(runtimeConfigBaseFolder, "inputschema.json"); var schemaFile = await RuntimeStorage.SaveFile(filePath, schema); var secretName = $"{config.Name}-inputschemafile"; - var schemaFileSecret = await KeyVaultClient.SaveSecretAsync(runtimeKeyVaultName, secretName, schemaFile); - flowToDeploy.SetStringToken(TokenName_InputSchemaFilePath, schemaFileSecret); + var schemaFileSecret = flowToDeploy.GetTokenString(PrepareSchemaFile.TokenName_InputSchemaFilePath); + await KeyVaultClient.SaveSecretAsync(schemaFileSecret, schemaFile); return "done"; } diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S650_FlattenJobConfig.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S650_FlattenJobConfig.cs index fd50cb2ef..0441150b5 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S650_FlattenJobConfig.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S650_FlattenJobConfig.cs @@ -3,14 +3,14 @@ // Licensed under the MIT License // ********************************************************************* using DataX.Config.ConfigDataModel; -using DataX.Config.ConfigGeneration.Processor; +using DataX.Config.ConfigDataModel.RuntimeConfig; using DataX.Config.Utility; using System; -using System.Collections.Generic; using System.Composition; using System.Linq; -using System.Text; +using System.Text.RegularExpressions; using System.Threading.Tasks; +using System.Globalization; namespace DataX.Config.ConfigGeneration.Processor { @@ -36,8 +36,6 @@ public override int GetOrder() public override async Task Process(FlowDeploymentSession flowToDeploy) { - var flowConfig = flowToDeploy.Config; - // get a flattener var flattener = await this.ConfigFlatteners.GetDefault(); if (flattener == null) @@ -54,19 +52,18 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) foreach(var job in jobs) { - var jsonContent = job.GetTokenString(GenerateJobConfig.TokenName_JobConfigContent); - var destFolder = job.GetTokenString(PrepareJobConfigVariables.TokenName_RuntimeConfigFolder); - - if (jsonContent != null) + foreach (var jc in job.JobConfigs) { - var json = JsonConfig.From(jsonContent); - job.SetStringToken(GenerateJobConfig.TokenName_JobConfigContent, flattener.Flatten(json)); + var jsonContent = job.Tokens.Resolve(jc.Content); - var destinationPath = ResourcePathUtil.Combine(destFolder, job.Name + ".conf"); - job.SetStringToken(GenerateJobConfig.TokenName_JobConfigFilePath, destinationPath); + if (jsonContent != null) + { + var json = JsonConfig.From(jsonContent); + jc.Content = flattener.Flatten(json); + } } } - + return "done"; } } diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S700_DeployJobConfigFile.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S700_DeployJobConfigFile.cs index b6e142770..cca815f99 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S700_DeployJobConfigFile.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S700_DeployJobConfigFile.cs @@ -39,16 +39,24 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) { // Deploy job configs var jobs = flowToDeploy.GetJobs(); - var deploymentTasks = jobs?.Select(async job => { - var content = job.GetTokenString(GenerateJobConfig.TokenName_JobConfigContent); - var filePath = job.GetTokenString(GenerateJobConfig.TokenName_JobConfigFilePath); - if (content != null && filePath != null) - { - job.SparkJobConfigFilePath = await this.JobData.SaveFile(filePath, content); - } - else + var deploymentTasks = jobs?.Select(async job => + { + + foreach (var jc in job.JobConfigs) { - job.SparkJobConfigFilePath = null; + var content = job.Tokens.Resolve(jc.Content); + var filePath = job.Tokens.Resolve(jc.FilePath); + if (content != null && filePath != null) + { + jc.SparkFilePath = await this.JobData.SaveFile(filePath, content); + job.SparkJobConfigFilePath = jc.SparkFilePath; + } + else + { + jc.SparkFilePath = null; + job.SparkJobConfigFilePath = null; + } + } return job.SparkJobConfigFilePath; diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S800_DeploySparkJob.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S800_DeploySparkJob.cs index 5b6f4418e..79d8dfedb 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S800_DeploySparkJob.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S800_DeploySparkJob.cs @@ -56,7 +56,8 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) // Ensure all creations are done successfully var upsertedJobNames = await Task.WhenAll(sparkJobUpsertions); - flowToDeploy.SetAttachment(AttachmentName_SparkJobNames, upsertedJobNames); + var jobNames = ParseJobNames(upsertedJobNames); + flowToDeploy.SetAttachment(AttachmentName_SparkJobNames, jobNames); return "done"; } @@ -86,29 +87,55 @@ public override async Task Delete(FlowDeploymentSession flowToDelete) /// create or update the spark job entry for the given job in deploying /// /// name of the spark job deployed - private async Task DeploySingleJob(JobDeploymentSession job, JsonConfig defaultSparkJobConfig) + private async Task DeploySingleJob(JobDeploymentSession job, JsonConfig defaultSparkJobConfig) { - // For each job - // replace config with tokens from job and flow, and the runtime config file path - // create spark job entry - var json = job.Tokens.Resolve(defaultSparkJobConfig); - var newJob = SparkJobConfig.From(json); - var jobName = newJob.Name; - var existingJob = await SparkJobData.GetByName(jobName); - - if (existingJob != null) + List names = new List(); + foreach (var jc in job.JobConfigs) { - //keep the state of the old job so we can stop that - newJob.SyncResult = existingJob.SyncResult; + // For each job + // replace config with tokens from job and flow, and the runtime config file path + // create spark job entry + job.SparkJobName = jc.SparkJobName; + job.SparkJobConfigFilePath = jc.SparkFilePath; + var json = job.Tokens.Resolve(defaultSparkJobConfig); + var newJob = SparkJobConfig.From(json); + var jobName = newJob.Name; + var existingJob = await SparkJobData.GetByName(jobName); + + if (existingJob != null) + { + //keep the state of the old job so we can stop that + newJob.SyncResult = existingJob.SyncResult; + } + + var result = await this.SparkJobData.UpsertByName(jobName, newJob); + if (!result.IsSuccess) + { + throw new ConfigGenerationException($"Failed to upsert into SparkJob table for job '{jobName}': {result.Message}"); + } + + names.Add(jobName); } - var result = await this.SparkJobData.UpsertByName(jobName, newJob); - if (!result.IsSuccess) + return names.ToArray(); + } + + /// + /// parse job names + /// + /// + /// + private string[] ParseJobNames(string[][] jobNames) + { + List names = new List(); + + for (int i = 0; i < jobNames.Length; i++) { - throw new ConfigGenerationException($"Failed to upsert into SparkJob table for job '{jobName}': {result.Message}"); + names.AddRange(jobNames[i].ToArray()); } - return jobName; + return names.ToArray(); } + } } diff --git a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S850_UpdateFlowMetrics.cs b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S850_UpdateFlowMetrics.cs index 0909495b5..9f4dd3c9c 100644 --- a/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S850_UpdateFlowMetrics.cs +++ b/Services/DataX.Config/DataX.Config/ConfigGeneration/Processor/S850_UpdateFlowMetrics.cs @@ -44,7 +44,7 @@ public override async Task Process(FlowDeploymentSession flowToDeploy) } // Get code gen data from earlier step - var rulesCode = flowToDeploy.GetAttachment(GenerateTransformFile.AttachmentName_CodeGenObject); + var rulesCode = flowToDeploy.GetAttachment(PrepareTransformFile.AttachmentName_CodeGenObject); Ensure.NotNull(rulesCode, "rulesCode"); Ensure.NotNull(rulesCode.MetricsRoot, "rulesCode.MetricsRoot"); Ensure.NotNull(rulesCode.MetricsRoot.metrics, "rulesCode.MetricsRoot.metrics"); diff --git a/Services/DataX.Config/DataX.Config/DataX.Config.csproj b/Services/DataX.Config/DataX.Config/DataX.Config.csproj index fbee157c9..17b35b180 100644 --- a/Services/DataX.Config/DataX.Config/DataX.Config.csproj +++ b/Services/DataX.Config/DataX.Config/DataX.Config.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true true FinalPublicKey.snk @@ -12,6 +12,7 @@ + pdbonly @@ -24,23 +25,24 @@ all runtime; build; native; contentfiles; analyzers - + all runtime; build; native; contentfiles; analyzers - + + - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Config/DataX.Config/InternalService/FlowDataManager.cs b/Services/DataX.Config/DataX.Config/InternalService/FlowDataManager.cs index 2375437d1..5fc358622 100644 --- a/Services/DataX.Config/DataX.Config/InternalService/FlowDataManager.cs +++ b/Services/DataX.Config/DataX.Config/InternalService/FlowDataManager.cs @@ -21,6 +21,11 @@ public class FlowDataManager public const string DataCollectionName = "flows"; public const string CommonDataName_DefaultFlowConfig = "defaultFlowConfig"; public const string CommonDataName_KafkaFlowConfig = "kafkaFlowConfig"; + public const string CommonDataName_BlobFlowConfig = "batchFlowConfig"; + + public const string GuiInputMode = "gui.input.mode"; + public const string GuiInputBatchingLastProcessedTime = "gui.input.batching.recurring.lastProcessedTime"; + [ImportingConstructor] public FlowDataManager(ConfigGenConfiguration configuration, IDesignTimeConfigStorage storage, ICommonDataManager commonsData) @@ -36,9 +41,16 @@ public FlowDataManager(ConfigGenConfiguration configuration, IDesignTimeConfigSt private string GetFlowConfigName(string type) { - if (!string.IsNullOrEmpty(type) && (type == Constants.InputType_Kafka || type == Constants.InputType_KafkaEventHub)) + if (!string.IsNullOrEmpty(type)) { - return CommonDataName_KafkaFlowConfig; + if (type == Constants.InputType_Kafka || type == Constants.InputType_KafkaEventHub) + { + return CommonDataName_KafkaFlowConfig; + } + else if (type == Constants.InputType_Blob) + { + return CommonDataName_BlobFlowConfig; + } } return CommonDataName_DefaultFlowConfig; @@ -50,6 +62,12 @@ public async Task GetByName(string flowName) return FlowConfig.From(json); } + public async Task GetByMode(string mode) + { + var jsons = await this.Storage.GetByFieldValue(mode, GuiInputMode, DataCollectionName).ConfigureAwait(false); + return jsons.Select(FlowConfig.From).ToArray(); + } + public async Task GetDefaultConfig(TokenDictionary tokens = null) { return await GetFlowConfigByInputType(Constants.InputType_EventHub, tokens); @@ -91,6 +109,12 @@ public async Task UpdateJobNamesForFlow(string flowName, string[] upsert return await this.Storage.UpdatePartialByName(json, FlowConfig.JsonFieldName_JobNames, flowName, DataCollectionName); } + public async Task UpdateLastProcessedTimeForFlow(string flowName, long lastProcessedTime) + { + var value = JsonConvert.SerializeObject(lastProcessedTime); + return await this.Storage.UpdatePartialByName(value, GuiInputBatchingLastProcessedTime, flowName, DataCollectionName).ConfigureAwait(false); + } + public async Task UpdateMetricsForFlow(string flowName, MetricsConfig metrics) { return await this.Storage.UpdatePartialByName(metrics.ToString(), FlowConfig.JsonFieldName_Metrics, flowName, DataCollectionName); diff --git a/Services/DataX.Config/DataX.Config/InternalService/JobDataManager.cs b/Services/DataX.Config/DataX.Config/InternalService/JobDataManager.cs index f278256f7..27f2c8953 100644 --- a/Services/DataX.Config/DataX.Config/InternalService/JobDataManager.cs +++ b/Services/DataX.Config/DataX.Config/InternalService/JobDataManager.cs @@ -63,5 +63,15 @@ public async Task DeleteConfigs(string destinationPath) { return await this.Storage.Delete(destinationPath); } + + /// + /// Delete the root folder containing all the job configs + /// + /// + /// + public async Task DeleteAll(string destinationPath) + { + return await this.Storage.DeleteAll(destinationPath); + } } } diff --git a/Services/DataX.Config/DataX.Config/InternalService/JobState.cs b/Services/DataX.Config/DataX.Config/InternalService/JobState.cs index 6960c0418..50b5bb457 100644 --- a/Services/DataX.Config/DataX.Config/InternalService/JobState.cs +++ b/Services/DataX.Config/DataX.Config/InternalService/JobState.cs @@ -9,6 +9,7 @@ public enum JobState Idle, Starting, Running, + Success, Error } } diff --git a/Services/DataX.Config/DataX.Config/InternalService/SparkClusterManager.cs b/Services/DataX.Config/DataX.Config/InternalService/SparkClusterManager.cs index d59b51e72..4d52f8b6c 100644 --- a/Services/DataX.Config/DataX.Config/InternalService/SparkClusterManager.cs +++ b/Services/DataX.Config/DataX.Config/InternalService/SparkClusterManager.cs @@ -37,9 +37,9 @@ public SparkClusterManager(SparkClusterData data, ISparkJobClientFactory clientF /// /// /// - public async Task GetSparkJobClient(string clusterName) + public async Task GetSparkJobClient(string clusterName, string databricksToken) { - return await CreateSparkJobClient(clusterName); + return await CreateSparkJobClient(clusterName, databricksToken); } /// @@ -47,7 +47,7 @@ public async Task GetSparkJobClient(string clusterName) /// /// /// - private async Task CreateSparkJobClient(string clusterName) + private async Task CreateSparkJobClient(string clusterName, string databricksToken) { var cluster = await this.ClusterData.GetByNameAsync(clusterName); if (cluster == null) @@ -56,6 +56,13 @@ private async Task CreateSparkJobClient(string clusterName) } var connectionString = await this.KeyVaultClient.ResolveSecretUriAsync(cluster.ConnectionString); + + //If databricks token exist then append it to the connection string + if(!string.IsNullOrWhiteSpace(databricksToken)) + { + connectionString += await this.KeyVaultClient.ResolveSecretUriAsync(databricksToken); + } + return await this.ClientFactory.GetClient(connectionString); } } diff --git a/Services/DataX.Config/DataX.Config/InternalService/SparkJobOperation.cs b/Services/DataX.Config/DataX.Config/InternalService/SparkJobOperation.cs index e093e8f76..38f871949 100644 --- a/Services/DataX.Config/DataX.Config/InternalService/SparkJobOperation.cs +++ b/Services/DataX.Config/DataX.Config/InternalService/SparkJobOperation.cs @@ -2,14 +2,12 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* -using Microsoft.Extensions.Logging; -using Newtonsoft.Json.Linq; using DataX.Config.ConfigDataModel; -using DataX.Config.Utility; using DataX.Contract; using DataX.Contract.Exception; +using Microsoft.Extensions.Logging; +using Newtonsoft.Json.Linq; using System; -using System.Collections.Generic; using System.Composition; using System.Linq; using System.Threading.Tasks; @@ -21,7 +19,7 @@ namespace DataX.Config public class SparkJobOperation { [ImportingConstructor] - public SparkJobOperation(SparkJobData jobData, SparkClusterManager clusterManager, ILogger logger) + public SparkJobOperation(SparkJobData jobData, SparkClusterManager clusterManager, ILogger logger) { this.JobData= jobData; this.ClusterManager = clusterManager; @@ -45,8 +43,8 @@ public async Task StartJob(string jobName) { Ensure.NotNull(jobName, "jobName"); Logger.LogInformation($"starting job '{jobName}'"); - var job = await EnsureJobState(jobName, JobState.Idle); - var sparkJobClient = await ClusterManager.GetSparkJobClient(job.Cluster); + var job = await EnsureJobState(jobName); + var sparkJobClient = await ClusterManager.GetSparkJobClient(job.Cluster, job.DatabricksToken); var result = await sparkJobClient.SubmitJob(job.Options); // Update job state @@ -83,10 +81,11 @@ public async Task StopJob(string jobName) { case JobState.Starting: case JobState.Running: - var sparkJobClient = await ClusterManager.GetSparkJobClient(job.Cluster); + var sparkJobClient = await ClusterManager.GetSparkJobClient(job.Cluster, job.DatabricksToken); var result = await sparkJobClient.StopJob(job.SyncResult.ClientCache); job.SyncResult = result; return await this.JobData.UpdateSyncResultByName(jobName, result); + case JobState.Success: case JobState.Idle: return new SuccessResult($"job '{jobName}' has already been stopped"); case JobState.Error: @@ -166,7 +165,7 @@ public async Task SyncJobState(string jobName) if (state.ClientCache != null && state.ClientCache.Type != JTokenType.Null) { - var sparkJobClient = await ClusterManager.GetSparkJobClient(job.Cluster); + var sparkJobClient = await ClusterManager.GetSparkJobClient(job.Cluster, job.DatabricksToken); var newResult = await sparkJobClient.GetJobInfo(state.ClientCache); state = newResult; Logger.LogInformation($"got state for job '{jobName}'"); @@ -224,14 +223,13 @@ public async Task SyncAllJobState() /// Make sure a job is in the specified state within /// /// job name - /// desired state /// retries to check for state /// interval in milliseconds to start perform next check /// the job config - private async Task EnsureJobState(string jobName, JobState state, int retries = _DefaultRetries, int intervalInMs = _DefaultIntervalInMs) + private async Task EnsureJobState(string jobName, int retries = _DefaultRetries, int intervalInMs = _DefaultIntervalInMs) { Ensure.NotNull(jobName, "jobName"); - Logger.LogInformation($"ensuring job '{jobName}' state to be '{state}'"); + Logger.LogInformation($"ensuring job '{jobName}' state to be idle or success"); int i = 0; while (i < retries) { @@ -239,7 +237,7 @@ private async Task EnsureJobState(string jobName, JobState state { //Get Job data var job = await SyncJobState(jobName); - if (job.SyncResult.JobState == state) + if (VerifyJobStopped(job.SyncResult.JobState)) { return job; } @@ -254,7 +252,17 @@ private async Task EnsureJobState(string jobName, JobState state i++; } - throw new GeneralException($"Job '{jobName}' failed to get to state '{state}' after '{retries}' sync attempts"); + throw new GeneralException($"Job '{jobName}' failed to get to state idle or success after '{retries}' sync attempts"); + } + + /// + /// Verify the job is not in running state + /// + /// job state + /// true if job is not running, false otherwise + public static bool VerifyJobStopped(JobState jobState) + { + return (jobState == JobState.Idle || jobState == JobState.Success || jobState == JobState.Error); } public async Task RetryInCaseOfException(Func> func, DateTime timeout, TimeSpan interval) diff --git a/Services/DataX.Config/DataX.Config/PublicService/FlowOperation.cs b/Services/DataX.Config/DataX.Config/PublicService/FlowOperation.cs index 19a3314e9..10eb8571c 100644 --- a/Services/DataX.Config/DataX.Config/PublicService/FlowOperation.cs +++ b/Services/DataX.Config/DataX.Config/PublicService/FlowOperation.cs @@ -69,6 +69,41 @@ public Task GetFlowByName(string name) return this.FlowData.GetByName(name); } + + /// + /// get flows by input mode + /// + /// + /// + public Task GetByMode(string mode) + { + return this.FlowData.GetByMode(mode); + } + + /// + /// Schedule batch jobs + /// + /// + /// + public async Task ScheduleBatch(RuntimeConfigGeneration runtimeConfigGeneration) + { + var flows = await GetByMode(Constants.InputMode_Batching).ConfigureAwait(false); + + Result result = null; + foreach (var flow in flows) + { + var name = flow.Name; + result = await runtimeConfigGeneration.GenerateRuntimeConfigs(name).ConfigureAwait(false); + + if (result.IsSuccess) + { + result = await RestartJobsForFlow(name).ConfigureAwait(false); + } + } + + return result ?? new SuccessResult("done"); + } + /// /// Save a flow with the gui input /// diff --git a/Services/DataX.Config/DataX.Config/PublicService/RuntimeConfigGeneration.cs b/Services/DataX.Config/DataX.Config/PublicService/RuntimeConfigGeneration.cs index 1f912eb86..2432e0f48 100644 --- a/Services/DataX.Config/DataX.Config/PublicService/RuntimeConfigGeneration.cs +++ b/Services/DataX.Config/DataX.Config/PublicService/RuntimeConfigGeneration.cs @@ -25,7 +25,7 @@ public class RuntimeConfigGeneration /// [ImportingConstructor] public RuntimeConfigGeneration( - ILogger logger, + ILogger logger, FlowDataManager flows, JobDataManager jobData, [ImportMany] IEnumerable flowProcessors diff --git a/Services/DataX.Config/DataX.Config/Storage/IRuntimeConfigStorage.cs b/Services/DataX.Config/DataX.Config/Storage/IRuntimeConfigStorage.cs index af9a5c48e..3671a4fa1 100644 --- a/Services/DataX.Config/DataX.Config/Storage/IRuntimeConfigStorage.cs +++ b/Services/DataX.Config/DataX.Config/Storage/IRuntimeConfigStorage.cs @@ -23,5 +23,14 @@ public interface IRuntimeConfigStorage /// Task Delete(string destinationPath); + + + /// + /// Deletes the destination files in folder + /// + /// path to file or folder to be deleted + /// + + Task DeleteAll(string destinationPath); } } diff --git a/Services/DataX.Config/DataX.Config/Utility/ConfigHelper.cs b/Services/DataX.Config/DataX.Config/Utility/ConfigHelper.cs new file mode 100644 index 000000000..975daa2c8 --- /dev/null +++ b/Services/DataX.Config/DataX.Config/Utility/ConfigHelper.cs @@ -0,0 +1,55 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Config.ConfigDataModel; +using System; +using System.Collections.Generic; +using System.Text; +using System.Text.RegularExpressions; + +namespace DataX.Config.Utility +{ + public static class ConfigHelper + { + /// + /// Parses the account name from connection string + /// + /// + /// account name + public static string ParseBlobAccountName(string connectionString) + { + string matched; + try + { + matched = Regex.Match(connectionString, @"(?<=AccountName=)(.*)(?=;AccountKey)").Value; + } + catch (Exception) + { + return "The connectionString does not have AccountName"; + } + + return matched; + } + + /// + /// Parses the account key from connection string + /// + /// + /// account key + public static string ParseBlobAccountKey(string connectionString) + { + string matched; + try + { + matched = Regex.Match(connectionString, @"(?<=AccountKey=)(.*)(?=;EndpointSuffix)").Value; + } + catch (Exception) + { + return "The connectionString does not have AccountKey"; + } + + return matched; + } + } +} diff --git a/Services/DataX.Config/DataX.Config/Utility/InstanceExportDescriptorProvider.cs b/Services/DataX.Config/DataX.Config/Utility/InstanceExportDescriptorProvider.cs deleted file mode 100644 index 99b82cda0..000000000 --- a/Services/DataX.Config/DataX.Config/Utility/InstanceExportDescriptorProvider.cs +++ /dev/null @@ -1,33 +0,0 @@ -// ********************************************************************* -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License -// ********************************************************************* -using System.Collections.Generic; -using System.Composition.Hosting.Core; - -namespace DataX.Config.Utility -{ - /// - /// Custom export descriptor provider class to add existing instances to MEF container - /// - public class InstanceExportDescriptorProvider : ExportDescriptorProvider - { - private readonly object[] _instances; - - public InstanceExportDescriptorProvider(object[] instances) - { - _instances = instances; - } - - public override IEnumerable GetExportDescriptors(CompositionContract contract, DependencyAccessor descriptorAccessor) - { - foreach (var instance in _instances) - { - if (contract.ContractType.IsInstanceOfType(instance)) - { - yield return new ExportDescriptorPromise(contract, contract.ContractType.FullName, true, NoDependencies, dependencies => ExportDescriptor.Create((context, operation) => instance, NoMetadata)); - } - } - } - } -} diff --git a/Services/DataX.Config/DataX.Config/Utility/KeyVaultUri.cs b/Services/DataX.Config/DataX.Config/Utility/KeyVaultUri.cs index acd8be144..7d7c1e1da 100644 --- a/Services/DataX.Config/DataX.Config/Utility/KeyVaultUri.cs +++ b/Services/DataX.Config/DataX.Config/Utility/KeyVaultUri.cs @@ -2,6 +2,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using DataX.Config.ConfigDataModel; using System; using System.Collections.Generic; using System.Text; @@ -11,11 +12,11 @@ namespace DataX.Config.Utility { public static class KeyVaultUri { - private static Regex _Reg = new Regex(@"^(keyvault:\/\/)+([^:\/\s]+)(\/)(.*)?", RegexOptions.IgnoreCase); + private static Regex _Reg = new Regex(@"^(keyvault:\/\/|secretscope:\/\/)+([^:\/\s]+)(\/)(.*)?", RegexOptions.IgnoreCase); - public static string ComposeUri(string keyvaultName, string secretName) + public static string ComposeUri(string keyvaultName, string secretName, string sparkType) { - return $"keyvault://{keyvaultName}/{secretName}"; + return (sparkType == ConfigDataModel.Constants.SparkTypeDataBricks) ? $"{Constants.PrefixSecretScope}://{keyvaultName}/{secretName}" : $"{Constants.PrefixKeyVault}://{keyvaultName}/{secretName}"; } public static bool IsSecretUri(string str) diff --git a/Services/DataX.Config/DataX.Config/Utility/LoggerExportDescriptorProvider.cs b/Services/DataX.Config/DataX.Config/Utility/LoggerExportDescriptorProvider.cs new file mode 100644 index 000000000..ed4f72f46 --- /dev/null +++ b/Services/DataX.Config/DataX.Config/Utility/LoggerExportDescriptorProvider.cs @@ -0,0 +1,78 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Utilities.Composition; +using Microsoft.Extensions.Logging; +using System; +using System.Collections.Generic; +using System.Composition.Hosting.Core; +using System.Linq; +using System.Reflection; + +namespace DataX.Config.Utility +{ + /// + /// Using a LoggerFactory, dynamically creates Logger to resolve for MEF + /// Falls back to InstanceExportDescriptorProvider implementation if not a Logger + /// + public class LoggerAndInstanceExportDescriptorProvider : InstanceExportDescriptorProvider + { + private static readonly Type _ILoggerType = typeof(ILogger); + + private static readonly MethodInfo _CreateLogger = + typeof(Microsoft.Extensions.Logging.LoggerFactoryExtensions) + .GetMethods() + .Where(m => m.Name == nameof(LoggerFactoryExtensions.CreateLogger) && m.IsGenericMethod) + .FirstOrDefault(); + + private readonly ILoggerFactory _loggerFactory; + private readonly bool _hasInstances; + + public LoggerAndInstanceExportDescriptorProvider(TValue instances, ILoggerFactory loggerFactory) + : base(instances) + { + object[] obj = (object[])Convert.ChangeType(instances, typeof(object[])); + _hasInstances = obj?.Length > 0; + _loggerFactory = loggerFactory; + } + + /// + public override IEnumerable GetExportDescriptors(CompositionContract contract, DependencyAccessor descriptorAccessor) + { + if (_loggerFactory != null && _ILoggerType.IsAssignableFrom(contract.ContractType)) + { + ILogger logger; + + if (contract.ContractType.GenericTypeArguments.Length > 0) + { + logger = CreateLogger(contract.ContractType.GenericTypeArguments.FirstOrDefault()); + } + else + { + logger = _loggerFactory.CreateLogger(contract.ContractType); + } + + yield return new ExportDescriptorPromise( + contract, + contract.ContractType.FullName, + true, + NoDependencies, + dependencies => ExportDescriptor.Create((context, operation) => logger, NoMetadata)); + } + else if(_hasInstances) + { + foreach (var descriptor in base.GetExportDescriptors(contract, descriptorAccessor)) + { + yield return descriptor; + } + } + } + + private ILogger CreateLogger(Type t) + { + var genericMethod = _CreateLogger.MakeGenericMethod(t); + return genericMethod.Invoke(null, new object[] { _loggerFactory }) as ILogger; + } + } +} diff --git a/Services/DataX.Config/DataX.Utility.EventHub/DataX.Utility.EventHub.csproj b/Services/DataX.Config/DataX.Utility.EventHub/DataX.Utility.EventHub.csproj index 868c7c313..a5626d212 100644 --- a/Services/DataX.Config/DataX.Utility.EventHub/DataX.Utility.EventHub.csproj +++ b/Services/DataX.Config/DataX.Utility.EventHub/DataX.Utility.EventHub.csproj @@ -1,7 +1,7 @@ - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -40,7 +40,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Contract/DataX.Contract.csproj b/Services/DataX.Contract/DataX.Contract.csproj index 00b4e89c7..1a31122d3 100644 --- a/Services/DataX.Contract/DataX.Contract.csproj +++ b/Services/DataX.Contract/DataX.Contract.csproj @@ -25,7 +25,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Contract/Settings/DataXSettings.cs b/Services/DataX.Contract/Settings/DataXSettings.cs new file mode 100644 index 000000000..07c825170 --- /dev/null +++ b/Services/DataX.Contract/Settings/DataXSettings.cs @@ -0,0 +1,37 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +namespace DataX.Contract.Settings +{ + /// + /// Used to hold basic settings for DataX. + /// + public class DataXSettings + { + // The flat settings should be better split into other settings classes + // For now, to keep compatibility with ServiceFabric AppManifest, this is flat + + public string CosmosDBConfigConnectionString { get; set; } + public string CosmosDBConfigDatabaseName { get; set; } + public string CosmosDBConfigCollectionName { get; set; } + + // DataX Settings + public bool EnableOneBox { get; set; } + public string LocalRoot { get; set; } + public string SparkHome { get; set; } + public string ClusterName { get; set; } + public string ServiceKeyVaultName { get; set; } + public string RuntimeKeyVaultName { get; set; } + public string MetricEventHubConnectionKey { get; set; } + public string ConfigFolderContainerPath { get; set; } + public string ConfigFolderHost { get; set; } + public string MetricsHttpEndpoint { get; set; } + public string AppInsightsIntrumentationKey { get; set; } + public string SparkType { get; set; } + + public string MefStorageAccountName { get; set; } + public string MefContainerName { get; set; } + public string MefBlobDirectory { get; set; } + } +} diff --git a/Services/DataX.Contract/Settings/DataXSettingsConstants.cs b/Services/DataX.Contract/Settings/DataXSettingsConstants.cs new file mode 100644 index 000000000..2184c31e9 --- /dev/null +++ b/Services/DataX.Contract/Settings/DataXSettingsConstants.cs @@ -0,0 +1,16 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +namespace DataX.Contract.Settings +{ + /// + /// Constants used with + /// + public static class DataXSettingsConstants + { + public const string DataX = nameof(DataX); + + public static readonly string ServiceEnvironment = $"{DataX}:{nameof(ServiceEnvironment)}"; + } +} diff --git a/Services/DataX.Flow/DataX.Flow.CodegenRules.Tests/DataX.Flow.CodegenRules.Tests.csproj b/Services/DataX.Flow/DataX.Flow.CodegenRules.Tests/DataX.Flow.CodegenRules.Tests.csproj index da66a945f..1a5529c0a 100644 --- a/Services/DataX.Flow/DataX.Flow.CodegenRules.Tests/DataX.Flow.CodegenRules.Tests.csproj +++ b/Services/DataX.Flow/DataX.Flow.CodegenRules.Tests/DataX.Flow.CodegenRules.Tests.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 false @@ -193,7 +193,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/DataX.Flow.CodegenRules/DataX.Flow.CodegenRules.csproj b/Services/DataX.Flow/DataX.Flow.CodegenRules/DataX.Flow.CodegenRules.csproj index 31ff96a18..5c42a22b3 100644 --- a/Services/DataX.Flow/DataX.Flow.CodegenRules/DataX.Flow.CodegenRules.csproj +++ b/Services/DataX.Flow/DataX.Flow.CodegenRules/DataX.Flow.CodegenRules.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -62,7 +62,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/DataX.Flow.Common.Tests/DataX.Flow.Common.Tests.csproj b/Services/DataX.Flow/DataX.Flow.Common.Tests/DataX.Flow.Common.Tests.csproj index 6ef2715f4..628474b9f 100644 --- a/Services/DataX.Flow/DataX.Flow.Common.Tests/DataX.Flow.Common.Tests.csproj +++ b/Services/DataX.Flow/DataX.Flow.Common.Tests/DataX.Flow.Common.Tests.csproj @@ -1,7 +1,7 @@ - netcoreapp2.1 + netcoreapp2.2 false @@ -41,7 +41,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/DataX.Flow.Common.Tests/GeneratorHelperTests.cs b/Services/DataX.Flow/DataX.Flow.Common.Tests/GeneratorHelperTests.cs new file mode 100644 index 000000000..2c9f0ee52 --- /dev/null +++ b/Services/DataX.Flow/DataX.Flow.Common.Tests/GeneratorHelperTests.cs @@ -0,0 +1,73 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Microsoft.VisualStudio.TestTools.UnitTesting; +using DataX.Flow.Common; +using DataX.Config.ConfigDataModel; + +namespace DataX.Flow.Generator.Tests +{ + [TestClass] + public class FlowGeneratorHelperTests + { + [TestMethod] + public void IsKeyVaultTest() + { + Assert.IsTrue(Helper.IsKeyVault("keyvault://abc")); + Assert.IsTrue(Helper.IsKeyVault("secretscope://abc")); + Assert.IsFalse(Helper.IsKeyVault("abc//keyvault://abc")); + Assert.IsFalse(Helper.IsKeyVault("abc.secretscope://abc")); + } + + [TestMethod] + public void ParseEventHubTest() + { + Assert.AreEqual("testeventhubname", Helper.ParseEventHub(@"Endpoint=sb://testnamespace.servicebus.windows.net/;SharedAccessKeyName=policyname;SharedAccessKey=12345=;EntityPath=testeventhubname")); + + Assert.AreEqual(null, Helper.ParseEventHub(@"Endpoint=sb://testnamespace.servicebus.windows.net/;SharedAccessKeyName=policyname;SharedAccessKey=12345")); + } + + [TestMethod] + public void ParseEventHubNamespaceTest() + { + Assert.AreEqual("testnamespace", Helper.ParseEventHubNamespace(@"Endpoint=sb://testnamespace.servicebus.windows.net/;SharedAccessKeyName=policyname;SharedAccessKey=12345=;EntityPath=testeventhubname")); + + Assert.AreEqual("testnamespace", Helper.ParseEventHubNamespace(@"Endpoint=sb://testnamespace.servicebus.windows.net/;SharedAccessKeyName=policyname;SharedAccessKey=12345")); + } + + [TestMethod] + public void ParseEventHubPolicyNameTest() + { + Assert.AreEqual("policyname", Helper.ParseEventHubPolicyName(@"Endpoint=sb://testnamespace.servicebus.windows.net/;SharedAccessKeyName=policyname;SharedAccessKey=12345=;EntityPath=testeventhubname")); + + Assert.AreEqual("policyname", Helper.ParseEventHubPolicyName(@"Endpoint =sb://testnamespace.servicebus.windows.net/;SharedAccessKeyName=policyname;SharedAccessKey=12345")); + } + + [TestMethod] + public void ParseEventHubAccessKeyTest() + { + Assert.AreEqual("12345=", Helper.ParseEventHubAccessKey(@"Endpoint=sb://testnamespace.servicebus.windows.net/;SharedAccessKeyName=policyname;SharedAccessKey=12345=;EntityPath=testeventhubname")); + + Assert.AreEqual("12345", Helper.ParseEventHubAccessKey(@"Endpoint =sb://testnamespace.servicebus.windows.net/;SharedAccessKeyName=policyname;SharedAccessKey=12345")); + } + + [TestMethod] + public void GetSecretFromKeyvaultIfNeededTest() + { + Assert.AreEqual("value", Helper.GetSecretFromKeyvaultIfNeeded(@"value")); + } + + [TestMethod] + public void GetKeyVaultNameTest() + { + Assert.AreEqual("keyvault://somekeyvalut/test-input-connectionstring-CD42404D52AD55CCFA9ACA4ADC828AA5", Helper.GetKeyVaultName("somekeyvalut", "test-input-connectionstring", Constants.SparkTypeHDInsight, "value")); + + Assert.AreEqual("keyvault://somekeyvalut/test-input-connectionstring", Helper.GetKeyVaultName("somekeyvalut", "test-input-connectionstring", Constants.SparkTypeHDInsight, "value", false)); + + Assert.AreEqual("secretscope://somekeyvalut/test-input-connectionstring-CD42404D52AD55CCFA9ACA4ADC828AA5", Helper.GetKeyVaultName("somekeyvalut", "test-input-connectionstring", Constants.SparkTypeDataBricks, "value")); + + Assert.AreEqual("secretscope://somekeyvalut/test-input-connectionstring", Helper.GetKeyVaultName("somekeyvalut", "test-input-connectionstring", Constants.SparkTypeDataBricks, "value", false)); + } + } +} diff --git a/Services/DataX.Flow/DataX.Flow.Common/DataX.Flow.Common.csproj b/Services/DataX.Flow/DataX.Flow.Common/DataX.Flow.Common.csproj index 81afc380d..36b597efe 100644 --- a/Services/DataX.Flow/DataX.Flow.Common/DataX.Flow.Common.csproj +++ b/Services/DataX.Flow/DataX.Flow.Common/DataX.Flow.Common.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -21,7 +21,7 @@ - + all runtime; build; native; contentfiles; analyzers @@ -44,7 +44,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/DataX.Flow.Common/EngineEnvironment.cs b/Services/DataX.Flow/DataX.Flow.Common/EngineEnvironment.cs index b57195e88..662fbbec0 100644 --- a/Services/DataX.Flow/DataX.Flow.Common/EngineEnvironment.cs +++ b/Services/DataX.Flow/DataX.Flow.Common/EngineEnvironment.cs @@ -3,11 +3,14 @@ // Licensed under the MIT License // ********************************************************************* using DataX.Contract; +using DataX.Contract.Settings; using DataX.Flow.Common.Models; +using DataX.ServiceHost; using DataX.ServiceHost.ServiceFabric; using DataX.Utilities.Blob; using DataX.Utilities.CosmosDB; using DataX.Utilities.KeyVault; +using Microsoft.Extensions.Configuration; using Newtonsoft.Json.Linq; using System; using System.Collections.Generic; @@ -34,6 +37,7 @@ public class EngineEnvironment public string OpsBlobConnectionString { get; set; } public SparkConnectionInfo SparkConnInfo { get; set; } public string SparkPassword { get; set; } + private readonly IConfiguration _configuration; /// /// OpsDiagnosticPath is the path that is used in c# code for calculating the garbage colection path for deleting the kernels @@ -56,6 +60,10 @@ public class EngineEnvironment public string OpsSparkSamplePath => $@"wasbs://samples@{EngineFlowConfig.OpsBlobBase}/"; public string Name { get; } + public EngineEnvironment(IConfiguration configuration) + { + _configuration = configuration; + } /// /// Moving the method that sets the various environment variables @@ -64,16 +72,29 @@ public class EngineEnvironment public async Task GetEnvironmentVariables() { CosmosDBDatabaseName = "production"; - - var response = ServiceFabricUtil.GetServiceKeyVaultName(); - if (response.Error.HasValue && response.Error.Value) - { - return ApiResult.CreateError(response.Message); + var cosmosCon = ""; + var cosmosDBCollectionName = ""; + var response = new ApiResult(); + string serviceKeyvaultName = string.Empty; + if (HostUtil.InServiceFabric) + { + response = ServiceFabricUtil.GetServiceKeyVaultName(); + if (response.Error.HasValue && response.Error.Value) + { + return ApiResult.CreateError(response.Message); + } + serviceKeyvaultName = response.Result.ToString(); + cosmosCon = KeyVault.GetSecretFromKeyvault(ServiceFabricUtil.GetServiceFabricConfigSetting("cosmosDBConfigConnectionString").Result.ToString()); + CosmosDBDatabaseName = KeyVault.GetSecretFromKeyvault(ServiceFabricUtil.GetServiceFabricConfigSetting("cosmosDBConfigDatabaseName").Result.ToString()); + cosmosDBCollectionName = ServiceFabricUtil.GetServiceFabricConfigSetting("cosmosDBConfigCollectionName").Result.ToString(); + } + else + { + serviceKeyvaultName = _configuration.GetSection(DataXSettingsConstants.ServiceEnvironment).GetSection(DataX.Config.ConfigDataModel.Constants.ConfigSettingName_ServiceKeyVaultName).Value; + cosmosCon = KeyVault.GetSecretFromKeyvault(_configuration.GetSection(DataXSettingsConstants.ServiceEnvironment).GetSection(DataX.Config.ConfigDataModel.Constants.ConfigSettingName_CosmosDBConfigConnectionString).Value); + CosmosDBDatabaseName = KeyVault.GetSecretFromKeyvault(_configuration.GetSection(DataXSettingsConstants.ServiceEnvironment).GetSection(DataX.Config.ConfigDataModel.Constants.ConfigSettingName_CosmosDBConfigDatabaseName).Value); + cosmosDBCollectionName = _configuration.GetSection(DataXSettingsConstants.ServiceEnvironment).GetSection(DataX.Config.ConfigDataModel.Constants.ConfigSettingName_CosmosDBConfigCollectionName).Value; } - string serviceKeyvaultName = response.Result.ToString(); - - var cosmosCon = KeyVault.GetSecretFromKeyvault(ServiceFabricUtil.GetServiceFabricConfigSetting("cosmosDBConfigConnectionString").Result.ToString()); - CosmosDBDatabaseName = KeyVault.GetSecretFromKeyvault(ServiceFabricUtil.GetServiceFabricConfigSetting("cosmosDBConfigDatabaseName").Result.ToString()); var namePassword = Helper.ParseCosmosDBUserNamePassword(cosmosCon); @@ -86,7 +107,7 @@ public async Task GetEnvironmentVariables() CosmosDBUserName = namePassword.Split(new char[] { ':' })[0]; CosmosDBPassword = namePassword.Split(new char[] { ':' })[1]; - response = await CosmosDB.DownloadConfigFromDocumentDB(CosmosDBDatabaseName, CosmosDBEndPoint, CosmosDBUserName, CosmosDBPassword, ServiceFabricUtil.GetServiceFabricConfigSetting("cosmosDBConfigCollectionName").Result.ToString()); + response = await CosmosDB.DownloadConfigFromDocumentDB(CosmosDBDatabaseName, CosmosDBEndPoint, CosmosDBUserName, CosmosDBPassword, cosmosDBCollectionName).ConfigureAwait(false); if (response.Error.HasValue && response.Error.Value) { return ApiResult.CreateError(response.Message); @@ -100,7 +121,10 @@ public async Task GetEnvironmentVariables() FlowBlobConnectionString = KeyVault.GetSecretFromKeyvault(serviceKeyvaultName, flowConfigObj.ConfiggenSecretPrefix + flowConfigObj.StorageAccountName + "-blobconnectionstring"); OpsBlobConnectionString = KeyVault.GetSecretFromKeyvault(serviceKeyvaultName, flowConfigObj.ConfiggenSecretPrefix + flowConfigObj.OpsStorageAccountName + "-blobconnectionstring"); - SparkConnInfo = Helper.ParseConnectionString(Helper.PathResolver(flowConfigObj.SparkConnectionString)); + if (EngineFlowConfig.SparkType != DataX.Config.ConfigDataModel.Constants.SparkTypeDataBricks) + { + SparkConnInfo = Helper.ParseConnectionString(Helper.PathResolver(flowConfigObj.SparkConnectionString)); + } return ApiResult.CreateSuccess(""); } diff --git a/Services/DataX.Flow/DataX.Flow.Common/Helper.cs b/Services/DataX.Flow/DataX.Flow.Common/Helper.cs index 1c1f99d69..d1000beb1 100644 --- a/Services/DataX.Flow/DataX.Flow.Common/Helper.cs +++ b/Services/DataX.Flow/DataX.Flow.Common/Helper.cs @@ -6,6 +6,7 @@ using DataX.Utilities.KeyVault; using System; using System.Collections.Generic; +using System.IO; using System.Linq; using System.Security.Cryptography; using System.Text; @@ -56,7 +57,7 @@ public static string GetSecretFromKeyvaultIfNeeded(string value) /// true if it is a secret, otherwise false public static bool IsKeyVault(string value) { - return value.StartsWith(GetKeyValutNamePrefix()); + return value.StartsWith(GetKeyValutNamePrefix()) || value.StartsWith(GetSecretScopePrefix()); } /// @@ -67,14 +68,14 @@ public static bool IsKeyVault(string value) /// /// /// keyVault uri - public static string GetKeyVaultName(string keyvaultName, string key, string value = "", bool postfix = true) + public static string GetKeyVaultName(string keyvaultName, string key, string sparkType, string value = "", bool postfix = true) { if (postfix) { key = key + $"-{Helper.GetHashCode(value)}"; } - return $"{GetKeyValutNamePrefix()}{keyvaultName}/{key}"; + return (sparkType == Config.ConfigDataModel.Constants.SparkTypeDataBricks) ? $"{GetSecretScopePrefix()}{keyvaultName}/{key}" : $"{GetKeyValutNamePrefix()}{keyvaultName}/{key}"; } /// @@ -86,9 +87,9 @@ public static string GetKeyVaultName(string keyvaultName, string key, string val /// /// /// secret name - public static string GenerateNewSecret(Dictionary keySecretList, string keyvaultName, string key, string value, bool postfix = true) + public static string GenerateNewSecret(Dictionary keySecretList, string keyvaultName, string key, string sparkType, string value, bool postfix = true) { - key = GetKeyVaultName(keyvaultName, key, value, postfix); + key = GetKeyVaultName(keyvaultName, key, sparkType, value, postfix); keySecretList.TryAdd(key, value); @@ -102,9 +103,17 @@ public static string GenerateNewSecret(Dictionary keySecretList, public static string GetKeyValutNamePrefix() { return "keyvault://"; - } - - + } + + /// + /// Get the prefix for secretscope uri + /// + /// the prefix for keyvault uri + public static string GetSecretScopePrefix() + { + return "secretscope://"; + } + /// /// Parses the eventhub namespace from connection string /// @@ -300,7 +309,7 @@ public static string PathResolver(string path) { if (path != null && Config.Utility.KeyVaultUri.IsSecretUri(path)) { - Regex r = new Regex(@"^((keyvault:?):\/\/)?([^:\/\s]+)(\/)(.*)?", RegexOptions.IgnoreCase); + Regex r = new Regex(@"^((keyvault|secretscope:?):\/\/)?([^:\/\s]+)(\/)(.*)?", RegexOptions.IgnoreCase); var keyvault = string.Empty; var secret = string.Empty; @@ -344,7 +353,47 @@ public static SparkConnectionInfo ParseConnectionString(string connectionString) UserName = match.Groups[2].Value, Password = match.Groups[3].Value }; - } + } + + /// + /// This method converts wasbs path to dbfs file path + /// + /// wasbs file path + /// file name + /// Returns dbfs file path + public static string ConvertToDbfsFilePath(string filePath, string fileName = "") + { + Regex opsPath = new Regex(@"wasbs:\/\/(.*)@(.*).blob.core.windows.net\/(.*)$", RegexOptions.IgnoreCase); + var match = opsPath.Match(filePath); + if (match.Success) + { + string result = Path.Combine(Config.ConfigDataModel.Constants.PrefixDbfs, Config.ConfigDataModel.Constants.PrefixDbfsMount + match.Groups[1].Value + "/", match.Groups[3].Value, fileName); + return result; + } + else + { + throw new Exception("Cannot convert to DBFS file path"); + } + } + + /// + /// This method returns a string value based on the spark type + /// + /// sparkType + /// Value to be used in case of HDInsight environment + /// Value to be used in case of Databricks environment + /// Returns string value based on spark type + public static string SetValueBasedOnSparkType(string sparkType, string valueForHDInsightEnv, string valueForDatabricksEnv) + { + if (sparkType != Config.ConfigDataModel.Constants.SparkTypeDataBricks) + { + return valueForHDInsightEnv; + } + else + { + return valueForDatabricksEnv; + } + } } /// diff --git a/Services/DataX.Flow/DataX.Flow.Common/Models/FlowConfigObject.cs b/Services/DataX.Flow/DataX.Flow.Common/Models/FlowConfigObject.cs index 27b73bf56..ec29c3bd3 100644 --- a/Services/DataX.Flow/DataX.Flow.Common/Models/FlowConfigObject.cs +++ b/Services/DataX.Flow/DataX.Flow.Common/Models/FlowConfigObject.cs @@ -74,6 +74,12 @@ public class FlowConfigObject [JsonProperty("configgenClientId")] public string ConfiggenClientId; + [JsonProperty("configgenClientSecret")] + public string ConfiggenClientSecret; + + [JsonProperty("configgenClientResourceId")] + public string ConfiggenClientResourceId; + [JsonProperty("configgenTenantId")] public string ConfiggenTenantId; diff --git a/Services/DataX.Flow/DataX.Flow.Common/Models/InteractiveQueryObject.cs b/Services/DataX.Flow/DataX.Flow.Common/Models/InteractiveQueryObject.cs index c666ee2fe..edb2f4278 100644 --- a/Services/DataX.Flow/DataX.Flow.Common/Models/InteractiveQueryObject.cs +++ b/Services/DataX.Flow/DataX.Flow.Common/Models/InteractiveQueryObject.cs @@ -2,6 +2,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using DataX.Config.ConfigDataModel; using Newtonsoft.Json; using System.Collections.Generic; @@ -48,6 +49,12 @@ public class InteractiveQueryObject [JsonProperty("inputType")] public string InputType; // "Event", "Iothub", "KafkaEventhub", "Kafka" + [JsonProperty("inputMode")] + public string InputMode; + + [JsonProperty("batchInputs")] + public List BatchInputs; + [JsonProperty("seconds")] public int Seconds; diff --git a/Services/DataX.Flow/DataX.Flow.Common/TemplateGenerator.cs b/Services/DataX.Flow/DataX.Flow.Common/TemplateGenerator.cs index 3a8505b0c..965ca6659 100644 --- a/Services/DataX.Flow/DataX.Flow.Common/TemplateGenerator.cs +++ b/Services/DataX.Flow/DataX.Flow.Common/TemplateGenerator.cs @@ -78,6 +78,7 @@ import org.apache.spark.sql.types._ val rawSchema = DataType.fromJson(rawSchemaString) +import org.apache.spark.sql.functions._ val file = ""<@SampleDataPath>"" val rawData = spark.read.json(file) rawData.select(from_json(col(""Raw""), rawSchema).alias(""Raw""), col(""Properties""), col(""SystemProperties"")).selectExpr(<@NormalizationSnippet>).createOrReplaceTempView(""DataXProcessedInput"") diff --git a/Services/DataX.Flow/DataX.Flow.DeleteHelper/ConfigDeleter.cs b/Services/DataX.Flow/DataX.Flow.DeleteHelper/ConfigDeleter.cs index 0cc66bc6d..fa1de121b 100644 --- a/Services/DataX.Flow/DataX.Flow.DeleteHelper/ConfigDeleter.cs +++ b/Services/DataX.Flow/DataX.Flow.DeleteHelper/ConfigDeleter.cs @@ -1,235 +1,239 @@ -// ********************************************************************* -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License -// ********************************************************************* -using Microsoft.Extensions.Logging; -using Newtonsoft.Json.Linq; -using DataX.Contract; -using DataX.Flow.Common; -using DataX.Flow.Common.Models; -using DataX.Utilities.Blob; -using DataX.Utilities.CosmosDB; -using DataX.Utilities.EventHub; -using DataX.Utilities.KeyVault; -using System; -using System.Collections.Generic; -using System.IO; -using System.Linq; -using System.Threading.Tasks; -using DataX.Config.ConfigDataModel; - -namespace DataX.Flow.DeleteHelper -{ - public class ConfigDeleter - { - private const string _Centralprocessing = "centralprocessing"; - private string _flowContainerName => _engineEnvironment.EngineFlowConfig.FlowContainerName; - private string _eventHubPrimaryKeyListener; - private EngineEnvironment _engineEnvironment = new EngineEnvironment(); - private readonly Dictionary _keySecretList = new Dictionary(); - - private string ConfigName { get; set; } - - private List _eventHubNames; - private string _eventHubNamespace; - private string _eventHubNameRole; - - private string ConsumerGroupName => ConfigName; - - private string ConsumerGroupNameDiagnostics => $"{ConfigName}-Diagnostic"; - - private string CodeGenFileName => $"{ConfigName}-combined.txt"; - - private string ProductConfigName => $"{ConfigName}-product.json"; - - private string JobConfigName => $"{ConfigName}-job.json"; - - private string ResourceConfigName => $"{ConfigName}-resources-changes.json"; - - private string OutputTemplateFileName => $"outputTemplates-{ConfigName}.xml"; - - private string OutputTemplatePath => $@"https://{_engineEnvironment.EngineFlowConfig.CPConfigFolderBase}/rules/outputTemplates"; - - private string RuleDefinitionFileName => $"rules-{ConfigName}.json"; - - private string RuleDefinitionPath => $@"https://{_engineEnvironment.EngineFlowConfig.CPConfigFolderBase}/rules/ruleDefinitions"; - - private string FlowContainerPath => $"https://{_engineEnvironment.EngineFlowConfig.CPConfigFolderBase}/{_flowContainerName}/{_engineEnvironment.EngineFlowConfig.EnvironmentType}/{ConfigName}/{ProductConfigName}"; - - public string CPConfigFolderPath => $"{_engineEnvironment.EngineFlowConfig.CPConfigFolderBase}/{_engineEnvironment.EngineFlowConfig.ContainerPath}/{ConfigName}"; - - public string SparkJobConfigFolderPath => $"{_engineEnvironment.EngineFlowConfig.SparkJobConfigFolderBase}/{_engineEnvironment.EngineFlowConfig.ContainerPath}"; - - private string ResourceConfigFolderPath => $"https://{_engineEnvironment.EngineFlowConfig.CPConfigFolderBase}/{_engineEnvironment.EngineFlowConfig.ResourceConfigPath}/{_engineEnvironment.EngineFlowConfig.ResourceConfigFileName}"; - - private string _inputEventhubConnectionStringRef; - private readonly ILogger _logger; - - public ConfigDeleter(ILogger logger) - { - _logger = logger; - } - - /// This is the function that can be called for deleting all assets created on save of a flow: consumergroup, secrets in Key Vault, cosmosDB products document such that this flow stops showing up in the UI under Flows and blobs - /// Please note if a a new asset is created on Azure as part of saving the flow or any other action taken by user in the UI, this function will need to be updated so that the asset can be deleted on delete of the flow. - /// - /// jObject requires: Subscription; Name; EventhubConnectionString; IsIotHub;EventhubName;userID - /// Returns result - success or failure as the case maybe - public async Task DeleteFlow(JObject jObject) - { - var diag = jObject.ToObject(); - - ConfigName = diag.Name; - bool errorExists = false; - var response = await _engineEnvironment.GetEnvironmentVariables(); - if (response.Error.HasValue && response.Error.Value) - { - _logger.LogError(response.Message); - return ApiResult.CreateError(response.Message); - } - - ///Delete consumer group - _logger.LogInformation($"For FlowId: {ConfigName} Deleting flow specific consumer group.. "); - var inputEventhubConnection = Helper.GetSecretFromKeyvaultIfNeeded(diag.EventhubConnectionString); - - _inputEventhubConnectionStringRef = Helper.IsKeyVault(diag.EventhubConnectionString) ? diag.EventhubConnectionString : Helper.GenerateNewSecret(_keySecretList, _engineEnvironment.EngineFlowConfig.SparkKeyVaultName, ConfigName + "-input-eventhubconnectionstring", diag.EventhubConnectionString, false); - diag.EventhubConnectionString = _inputEventhubConnectionStringRef; - - if (diag.InputType == Constants.InputType_EventHub) - { - var ehName = Helper.ParseEventHub(inputEventhubConnection); - _eventHubNamespace = Helper.ParseEventHubNamespace(inputEventhubConnection); - _eventHubNameRole = Helper.ParseEventHubPolicyName(inputEventhubConnection); - _eventHubPrimaryKeyListener = Helper.ParseEventHubAccessKey(inputEventhubConnection); - - if (string.IsNullOrWhiteSpace(ehName) || string.IsNullOrWhiteSpace(_eventHubNamespace) || string.IsNullOrWhiteSpace(_eventHubNameRole) || string.IsNullOrWhiteSpace(_eventHubPrimaryKeyListener)) - { - string error = "The connection string for Event Hub input type must contain Endpoint, SharedAccessKeyName, SharedAccessKey, and EntityPath"; - _logger.LogError(error); - errorExists = true; - } - - _eventHubNames = new List() { ehName }; - } - else - { - _eventHubNames = Helper.ParseEventHubNames(diag.EventhubNames); - _eventHubNamespace = Helper.ParseEventHubNamespace(inputEventhubConnection); - _eventHubNameRole = Helper.ParseEventHubPolicyName(inputEventhubConnection); - _eventHubPrimaryKeyListener = Helper.ParseEventHubAccessKey(inputEventhubConnection); - - if (_eventHubNames.Count < 1) - { - string error = "The event hub-compatible name for IoT Hub input type must be defined"; - _logger.LogError(error); - errorExists = true; - } - - if (string.IsNullOrWhiteSpace(_eventHubNamespace) || string.IsNullOrWhiteSpace(_eventHubNameRole) || string.IsNullOrWhiteSpace(_eventHubPrimaryKeyListener)) - { - string error = "The event hub-compatible endpoint for IoT Hub input type must contain Endpoint, SharedAccessKeyName, and SharedAccessKey"; - _logger.LogError(error); - errorExists = true; - } - } - - // ResourceCreation is one of the environment variables. - // If you don't want to create resource, you can set this to false. - if (_engineEnvironment.ResourceCreation && (diag.InputType == Constants.InputType_EventHub || diag.InputType == Constants.InputType_IoTHub)) - { - var inputSubscriptionId = string.IsNullOrEmpty(diag.InputSubscriptionId) ? Helper.GetSecretFromKeyvaultIfNeeded(_engineEnvironment.EngineFlowConfig.SubscriptionId) : Helper.GetSecretFromKeyvaultIfNeeded(diag.InputSubscriptionId); - var inputResourceGroup = string.IsNullOrEmpty(diag.InputResourceGroup) ? _engineEnvironment.EngineFlowConfig.EventHubResourceGroupName : Helper.GetSecretFromKeyvaultIfNeeded(diag.InputResourceGroup); - - foreach (string ehName in _eventHubNames) - { - var result = EventHub.DeleteConsumerGroup(inputSubscriptionId, _engineEnvironment.EngineFlowConfig.ServiceKeyVaultName, inputResourceGroup, _engineEnvironment.EngineFlowConfig.EventHubResourceGroupLocation, _eventHubNamespace, ehName, ConsumerGroupName, diag.InputType, _engineEnvironment.EngineFlowConfig.ConfiggenClientId, _engineEnvironment.EngineFlowConfig.ConfiggenTenantId, _engineEnvironment.EngineFlowConfig.ConfiggenSecretPrefix); - if (result.Error.HasValue && result.Error.Value) - { - _logger.LogError(result.Message); - errorExists = true; - } - else - { - _logger.LogInformation($"For FlowId: {ConfigName} Successfully deleted flow specific consumer group"); - } - } - } - - - ///Delete cosmosDB document related to a flow - response = await CosmosDB.DeleteConfigFromDocumentDB(_engineEnvironment.CosmosDBDatabaseName, _engineEnvironment.CosmosDBEndPoint, _engineEnvironment.CosmosDBUserName, _engineEnvironment.CosmosDBPassword, "flows", ConfigName); - if (response.Error.HasValue && response.Error.Value) - { - _logger.LogError(response.Message); - errorExists = true; - } - else - { - _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific cosmosDB entry"); - } - - ///Delete configs stored in blobs - // ruleDefinitions - response = await BlobHelper.DeleteBlob(_engineEnvironment.FlowBlobConnectionString, Path.Combine(RuleDefinitionPath, RuleDefinitionFileName)); - if (response.Error.HasValue && response.Error.Value) - { - _logger.LogError(response.Message); - errorExists = true; - } - else - { - _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific rules definition blob"); - } - - // outputTemplates - response = await BlobHelper.DeleteBlob(_engineEnvironment.FlowBlobConnectionString, Path.Combine(OutputTemplatePath, OutputTemplateFileName)); - if (response.Error.HasValue && response.Error.Value) - { - _logger.LogError(response.Message); - errorExists = true; - } - else - { - _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific output template blob"); - } - - string resourceGroupLocation = _engineEnvironment.EngineFlowConfig.ResourceGroupLocation; - string resourceGroupName = _engineEnvironment.EngineFlowConfig.ResourceGroupName; - string storageAccountName = _engineEnvironment.EngineFlowConfig.StorageAccountName; - string containerPath = Path.Combine(_flowContainerName, _engineEnvironment.EngineFlowConfig.EnvironmentType, ConfigName); - string subscriptionId = Helper.GetSecretFromKeyvaultIfNeeded(_engineEnvironment.EngineFlowConfig.SubscriptionId); - - BlobStorage.DeleteAllConfigsFromBlobStorage(subscriptionId, _engineEnvironment.EngineFlowConfig.ServiceKeyVaultName, resourceGroupName, resourceGroupLocation, storageAccountName, _Centralprocessing, Path.Combine(_engineEnvironment.EngineFlowConfig.ContainerPath, ConfigName), _engineEnvironment.EngineFlowConfig.ConfiggenClientId, _engineEnvironment.EngineFlowConfig.ConfiggenTenantId, _engineEnvironment.EngineFlowConfig.ConfiggenSecretPrefix); - _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific blobs under the folder {ConfigName} under container {_Centralprocessing}"); - - BlobStorage.DeleteAllConfigsFromBlobStorage(subscriptionId, _engineEnvironment.EngineFlowConfig.ServiceKeyVaultName, resourceGroupName, resourceGroupLocation, storageAccountName, _Centralprocessing, Path.Combine(_engineEnvironment.EngineFlowConfig.ContainerPath, ConfigName), _engineEnvironment.EngineFlowConfig.ConfiggenClientId, _engineEnvironment.EngineFlowConfig.ConfiggenTenantId, _engineEnvironment.EngineFlowConfig.ConfiggenSecretPrefix); - - BlobStorage.DeleteAllConfigsFromBlobStorage(subscriptionId, _engineEnvironment.EngineFlowConfig.ServiceKeyVaultName, resourceGroupName, resourceGroupLocation, storageAccountName, _flowContainerName, Path.Combine(_engineEnvironment.EngineFlowConfig.EnvironmentType, ConfigName), _engineEnvironment.EngineFlowConfig.ConfiggenClientId, _engineEnvironment.EngineFlowConfig.ConfiggenTenantId, _engineEnvironment.EngineFlowConfig.ConfiggenSecretPrefix); - - _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific productconfig: {ProductConfigName} and {JobConfigName} for {ConfigName}."); - - /// Delete sample data and the checkpoints folder if it exists for that flow - var hashValue = Helper.GetHashCode(diag.UserName); - await BlobHelper.DeleteBlob(_engineEnvironment.OpsBlobConnectionString, Path.Combine(_engineEnvironment.OpsSamplePath, $"{ConfigName}-{hashValue}.json")); - _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific sampledata file: {ConfigName}-{ hashValue}.json"); - - await BlobHelper.DeleteAllBlobsInAContainer(_engineEnvironment.OpsBlobConnectionString, $"{_engineEnvironment.CheckPointContainerNameHelper(ConfigName)}-checkpoints", _engineEnvironment.EngineFlowConfig.OpsBlobDirectory); - _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific checkpoints for {ConfigName}."); - - _logger.LogInformation("Deleting flow specific secrets.."); - ///Delete secrets specific to a flow from KeyVault - KeyVault.GetSecretsAndDeleteFromKeyvault(_engineEnvironment.EngineFlowConfig.SparkKeyVaultName, ConfigName); - _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific secrets"); - - if (!errorExists) - { - return ApiResult.CreateSuccess("Deleted!"); - } - else - { - return ApiResult.CreateError("Deleted but with some error. Please check logs for details"); - } - } - - } -} +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Microsoft.Extensions.Logging; +using Newtonsoft.Json.Linq; +using DataX.Contract; +using DataX.Flow.Common; +using DataX.Flow.Common.Models; +using DataX.Utilities.Blob; +using DataX.Utilities.CosmosDB; +using DataX.Utilities.EventHub; +using DataX.Utilities.KeyVault; +using System; +using System.Collections.Generic; +using System.IO; +using System.Linq; +using System.Threading.Tasks; +using Microsoft.Extensions.Configuration; +using DataX.Config.ConfigDataModel; + +namespace DataX.Flow.DeleteHelper +{ + public class ConfigDeleter + { + private const string _Centralprocessing = "centralprocessing"; + private string _flowContainerName => _engineEnvironment.EngineFlowConfig.FlowContainerName; + private string _eventHubPrimaryKeyListener; + private EngineEnvironment _engineEnvironment; + private readonly Dictionary _keySecretList = new Dictionary(); + + private string ConfigName { get; set; } + + private List _eventHubNames; + private string _eventHubNamespace; + private string _eventHubNameRole; + + private string ConsumerGroupName => ConfigName; + + private string ConsumerGroupNameDiagnostics => $"{ConfigName}-Diagnostic"; + + private string CodeGenFileName => $"{ConfigName}-combined.txt"; + + private string ProductConfigName => $"{ConfigName}-product.json"; + + private string JobConfigName => $"{ConfigName}-job.json"; + + private string ResourceConfigName => $"{ConfigName}-resources-changes.json"; + + private string OutputTemplateFileName => $"outputTemplates-{ConfigName}.xml"; + + private string OutputTemplatePath => $@"https://{_engineEnvironment.EngineFlowConfig.CPConfigFolderBase}/rules/outputTemplates"; + + private string RuleDefinitionFileName => $"rules-{ConfigName}.json"; + + private string RuleDefinitionPath => $@"https://{_engineEnvironment.EngineFlowConfig.CPConfigFolderBase}/rules/ruleDefinitions"; + + private string FlowContainerPath => $"https://{_engineEnvironment.EngineFlowConfig.CPConfigFolderBase}/{_flowContainerName}/{_engineEnvironment.EngineFlowConfig.EnvironmentType}/{ConfigName}/{ProductConfigName}"; + + public string CPConfigFolderPath => $"{_engineEnvironment.EngineFlowConfig.CPConfigFolderBase}/{_engineEnvironment.EngineFlowConfig.ContainerPath}/{ConfigName}"; + + public string SparkJobConfigFolderPath => $"{_engineEnvironment.EngineFlowConfig.SparkJobConfigFolderBase}/{_engineEnvironment.EngineFlowConfig.ContainerPath}"; + + private string ResourceConfigFolderPath => $"https://{_engineEnvironment.EngineFlowConfig.CPConfigFolderBase}/{_engineEnvironment.EngineFlowConfig.ResourceConfigPath}/{_engineEnvironment.EngineFlowConfig.ResourceConfigFileName}"; + + private string _inputEventhubConnectionStringRef; + private readonly ILogger _logger; + private readonly IConfiguration _configuration; + + public ConfigDeleter(ILogger logger, IConfiguration configuration) + { + _logger = logger; + _configuration = configuration; + _engineEnvironment = new EngineEnvironment(_configuration); + } + + /// This is the function that can be called for deleting all assets created on save of a flow: consumergroup, secrets in Key Vault, cosmosDB products document such that this flow stops showing up in the UI under Flows and blobs + /// Please note if a a new asset is created on Azure as part of saving the flow or any other action taken by user in the UI, this function will need to be updated so that the asset can be deleted on delete of the flow. + /// + /// jObject requires: Subscription; Name; EventhubConnectionString; IsIotHub;EventhubName;userID + /// Returns result - success or failure as the case maybe + public async Task DeleteFlow(JObject jObject) + { + var diag = jObject.ToObject(); + + ConfigName = diag.Name; + bool errorExists = false; + var response = await _engineEnvironment.GetEnvironmentVariables(); + if (response.Error.HasValue && response.Error.Value) + { + _logger.LogError(response.Message); + return ApiResult.CreateError(response.Message); + } + + ///Delete consumer group + _logger.LogInformation($"For FlowId: {ConfigName} Deleting flow specific consumer group.. "); + var inputEventhubConnection = Helper.GetSecretFromKeyvaultIfNeeded(diag.EventhubConnectionString); + + _inputEventhubConnectionStringRef = Helper.IsKeyVault(diag.EventhubConnectionString) ? diag.EventhubConnectionString : Helper.GenerateNewSecret(_keySecretList, _engineEnvironment.EngineFlowConfig.SparkKeyVaultName, ConfigName + "-input-eventhubconnectionstring", _engineEnvironment.EngineFlowConfig.SparkType, diag.EventhubConnectionString, false); + diag.EventhubConnectionString = _inputEventhubConnectionStringRef; + + if (diag.InputType == Constants.InputType_EventHub && !string.IsNullOrEmpty(inputEventhubConnection)) + { + var ehName = Helper.ParseEventHub(inputEventhubConnection); + _eventHubNamespace = Helper.ParseEventHubNamespace(inputEventhubConnection); + _eventHubNameRole = Helper.ParseEventHubPolicyName(inputEventhubConnection); + _eventHubPrimaryKeyListener = Helper.ParseEventHubAccessKey(inputEventhubConnection); + + if (string.IsNullOrWhiteSpace(ehName) || string.IsNullOrWhiteSpace(_eventHubNamespace) || string.IsNullOrWhiteSpace(_eventHubNameRole) || string.IsNullOrWhiteSpace(_eventHubPrimaryKeyListener)) + { + string error = "The connection string for Event Hub input type must contain Endpoint, SharedAccessKeyName, SharedAccessKey, and EntityPath"; + _logger.LogError(error); + errorExists = true; + } + + _eventHubNames = new List() { ehName }; + } + else if (diag.InputType == Constants.InputType_IoTHub && !string.IsNullOrEmpty(diag.EventhubNames) && !string.IsNullOrEmpty(inputEventhubConnection)) + { + _eventHubNames = Helper.ParseEventHubNames(diag.EventhubNames); + _eventHubNamespace = Helper.ParseEventHubNamespace(inputEventhubConnection); + _eventHubNameRole = Helper.ParseEventHubPolicyName(inputEventhubConnection); + _eventHubPrimaryKeyListener = Helper.ParseEventHubAccessKey(inputEventhubConnection); + + if (_eventHubNames.Count < 1) + { + string error = "The event hub-compatible name for IoT Hub input type must be defined"; + _logger.LogError(error); + errorExists = true; + } + + if (string.IsNullOrWhiteSpace(_eventHubNamespace) || string.IsNullOrWhiteSpace(_eventHubNameRole) || string.IsNullOrWhiteSpace(_eventHubPrimaryKeyListener)) + { + string error = "The event hub-compatible endpoint for IoT Hub input type must contain Endpoint, SharedAccessKeyName, and SharedAccessKey"; + _logger.LogError(error); + errorExists = true; + } + } + + // ResourceCreation is one of the environment variables. + // If you don't want to create resource, you can set this to false. + if (_engineEnvironment.ResourceCreation && _eventHubNames != null && (diag.InputType == Constants.InputType_EventHub || diag.InputType == Constants.InputType_IoTHub)) + { + var inputSubscriptionId = string.IsNullOrEmpty(diag.InputSubscriptionId) ? Helper.GetSecretFromKeyvaultIfNeeded(_engineEnvironment.EngineFlowConfig.SubscriptionId) : Helper.GetSecretFromKeyvaultIfNeeded(diag.InputSubscriptionId); + var inputResourceGroup = string.IsNullOrEmpty(diag.InputResourceGroup) ? _engineEnvironment.EngineFlowConfig.EventHubResourceGroupName : Helper.GetSecretFromKeyvaultIfNeeded(diag.InputResourceGroup); + + foreach (string ehName in _eventHubNames) + { + var result = EventHub.DeleteConsumerGroup(inputSubscriptionId, _engineEnvironment.EngineFlowConfig.ServiceKeyVaultName, inputResourceGroup, _engineEnvironment.EngineFlowConfig.EventHubResourceGroupLocation, _eventHubNamespace, ehName, ConsumerGroupName, diag.InputType, _engineEnvironment.EngineFlowConfig.ConfiggenClientId, _engineEnvironment.EngineFlowConfig.ConfiggenTenantId, _engineEnvironment.EngineFlowConfig.ConfiggenSecretPrefix); + if (result.Error.HasValue && result.Error.Value) + { + _logger.LogError(result.Message); + errorExists = true; + } + else + { + _logger.LogInformation($"For FlowId: {ConfigName} Successfully deleted flow specific consumer group"); + } + } + } + + + ///Delete cosmosDB document related to a flow + response = await CosmosDB.DeleteConfigFromDocumentDB(_engineEnvironment.CosmosDBDatabaseName, _engineEnvironment.CosmosDBEndPoint, _engineEnvironment.CosmosDBUserName, _engineEnvironment.CosmosDBPassword, "flows", ConfigName); + if (response.Error.HasValue && response.Error.Value) + { + _logger.LogError(response.Message); + errorExists = true; + } + else + { + _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific cosmosDB entry"); + } + + ///Delete configs stored in blobs + // ruleDefinitions + response = await BlobHelper.DeleteBlob(_engineEnvironment.FlowBlobConnectionString, Path.Combine(RuleDefinitionPath, RuleDefinitionFileName)); + if (response.Error.HasValue && response.Error.Value) + { + _logger.LogError(response.Message); + errorExists = true; + } + else + { + _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific rules definition blob"); + } + + // outputTemplates + response = await BlobHelper.DeleteBlob(_engineEnvironment.FlowBlobConnectionString, Path.Combine(OutputTemplatePath, OutputTemplateFileName)); + if (response.Error.HasValue && response.Error.Value) + { + _logger.LogError(response.Message); + errorExists = true; + } + else + { + _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific output template blob"); + } + + string resourceGroupLocation = _engineEnvironment.EngineFlowConfig.ResourceGroupLocation; + string resourceGroupName = _engineEnvironment.EngineFlowConfig.ResourceGroupName; + string storageAccountName = _engineEnvironment.EngineFlowConfig.StorageAccountName; + string containerPath = Path.Combine(_flowContainerName, _engineEnvironment.EngineFlowConfig.EnvironmentType, ConfigName); + string subscriptionId = Helper.GetSecretFromKeyvaultIfNeeded(_engineEnvironment.EngineFlowConfig.SubscriptionId); + + BlobStorage.DeleteAllConfigsFromBlobStorage(subscriptionId, _engineEnvironment.EngineFlowConfig.ServiceKeyVaultName, resourceGroupName, resourceGroupLocation, storageAccountName, _Centralprocessing, Path.Combine(_engineEnvironment.EngineFlowConfig.ContainerPath, ConfigName), _engineEnvironment.EngineFlowConfig.ConfiggenClientId, _engineEnvironment.EngineFlowConfig.ConfiggenTenantId, _engineEnvironment.EngineFlowConfig.ConfiggenSecretPrefix); + _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific blobs under the folder {ConfigName} under container {_Centralprocessing}"); + + BlobStorage.DeleteAllConfigsFromBlobStorage(subscriptionId, _engineEnvironment.EngineFlowConfig.ServiceKeyVaultName, resourceGroupName, resourceGroupLocation, storageAccountName, _Centralprocessing, Path.Combine(_engineEnvironment.EngineFlowConfig.ContainerPath, ConfigName), _engineEnvironment.EngineFlowConfig.ConfiggenClientId, _engineEnvironment.EngineFlowConfig.ConfiggenTenantId, _engineEnvironment.EngineFlowConfig.ConfiggenSecretPrefix); + + BlobStorage.DeleteAllConfigsFromBlobStorage(subscriptionId, _engineEnvironment.EngineFlowConfig.ServiceKeyVaultName, resourceGroupName, resourceGroupLocation, storageAccountName, _flowContainerName, Path.Combine(_engineEnvironment.EngineFlowConfig.EnvironmentType, ConfigName), _engineEnvironment.EngineFlowConfig.ConfiggenClientId, _engineEnvironment.EngineFlowConfig.ConfiggenTenantId, _engineEnvironment.EngineFlowConfig.ConfiggenSecretPrefix); + + _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific productconfig: {ProductConfigName} and {JobConfigName} for {ConfigName}."); + + /// Delete sample data and the checkpoints folder if it exists for that flow + var hashValue = Helper.GetHashCode(diag.UserName); + await BlobHelper.DeleteBlob(_engineEnvironment.OpsBlobConnectionString, Path.Combine(_engineEnvironment.OpsSamplePath, $"{ConfigName}-{hashValue}.json")); + _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific sampledata file: {ConfigName}-{ hashValue}.json"); + + await BlobHelper.DeleteAllBlobsInAContainer(_engineEnvironment.OpsBlobConnectionString, $"{_engineEnvironment.CheckPointContainerNameHelper(ConfigName)}-checkpoints", _engineEnvironment.EngineFlowConfig.OpsBlobDirectory); + _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific checkpoints for {ConfigName}."); + + _logger.LogInformation("Deleting flow specific secrets.."); + ///Delete secrets specific to a flow from KeyVault + KeyVault.GetSecretsAndDeleteFromKeyvault(_engineEnvironment.EngineFlowConfig.SparkKeyVaultName, ConfigName); + _logger.LogInformation($"For FlowId: {ConfigName} Successfully Deleted flow specific secrets"); + + if (!errorExists) + { + return ApiResult.CreateSuccess("Deleted!"); + } + else + { + return ApiResult.CreateError("Deleted but with some error. Please check logs for details"); + } + } + + } +} diff --git a/Services/DataX.Flow/DataX.Flow.DeleteHelper/DataX.Flow.DeleteHelper.csproj b/Services/DataX.Flow/DataX.Flow.DeleteHelper/DataX.Flow.DeleteHelper.csproj index 527b6e3a6..022a27077 100644 --- a/Services/DataX.Flow/DataX.Flow.DeleteHelper/DataX.Flow.DeleteHelper.csproj +++ b/Services/DataX.Flow/DataX.Flow.DeleteHelper/DataX.Flow.DeleteHelper.csproj @@ -1,7 +1,7 @@ - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -37,7 +37,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/DataX.Flow.InteractiveQuery.Tests/DataX.Flow.InteractiveQuery.Tests.csproj b/Services/DataX.Flow/DataX.Flow.InteractiveQuery.Tests/DataX.Flow.InteractiveQuery.Tests.csproj new file mode 100644 index 000000000..636555a8a --- /dev/null +++ b/Services/DataX.Flow/DataX.Flow.InteractiveQuery.Tests/DataX.Flow.InteractiveQuery.Tests.csproj @@ -0,0 +1,19 @@ + + + + netcoreapp2.2 + + false + + + + + + + + + + + + + diff --git a/Services/DataX.Flow/DataX.Flow.InteractiveQuery.Tests/InteractiveQueryTests.cs b/Services/DataX.Flow/DataX.Flow.InteractiveQuery.Tests/InteractiveQueryTests.cs new file mode 100644 index 000000000..4eef25f4b --- /dev/null +++ b/Services/DataX.Flow/DataX.Flow.InteractiveQuery.Tests/InteractiveQueryTests.cs @@ -0,0 +1,103 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Flow.Common; +using Microsoft.VisualStudio.TestTools.UnitTesting; + +namespace DataX.Flow.InteractiveQuery.Tests +{ + [TestClass] + public class InteractiveQueryTests + { + [TestMethod] + public void ConvertWasbsToDbfsFilePath() + { + //Test ConvertToDbfsFilePath with both filePath and fileName parameters + string wasbsPath = "wasbs://mycontainer@mystorageaccount.blob.core.windows.net/"; + string fileName = "testFile.json"; + string actualValue = Helper.ConvertToDbfsFilePath(wasbsPath, fileName); + string expectedValue = "dbfs:/mnt/livequery/mycontainer/testFile.json"; + Assert.AreEqual(expectedValue, actualValue, "DBFS file path is incorrect"); + + //Test ConvertToDbfsFilePath with only filePath parameter + wasbsPath = "wasbs://mycontainer@mystorageaccount.blob.core.windows.net/testfolder/testFile.json"; + actualValue = Helper.ConvertToDbfsFilePath(wasbsPath); + expectedValue = "dbfs:/mnt/livequery/mycontainer/testfolder/testFile.json"; + Assert.AreEqual(expectedValue, actualValue, "DBFS file path is incorrect"); + } + + [TestMethod] + public void TestMountCode() + { + string expectedValue = "dbutils.fs.mount(source = \"wasbs://mycontainer@teststorageaccount.blob.core.windows.net/\", mountPoint = \"/mnt/livequery//mycontainer\", extraConfigs = Map(\"fs.azure.account.key.teststorageaccount.blob.core.windows.net\"->dbutils.secrets.get(scope = \"testkeyvault\", key = \"datax-sa-teststorageaccount\")))"; + + //Test unnested file path + string dbfsPath = "dbfs:/mnt/livequery/mycontainer/testFile.json"; + string opsStorageAccountName = "teststorageaccount"; + string sparkKeyVaultName = "testkeyvault"; + string actualValue = KernelService.CreateMountCode(dbfsPath, opsStorageAccountName, sparkKeyVaultName); + Assert.AreEqual(expectedValue, actualValue, "Mount code is incorrect"); + + //Test nested file path + dbfsPath = "dbfs:/mnt/livequery/mycontainer/folder1/folder2/testFile.json"; + actualValue = KernelService.CreateMountCode(dbfsPath, opsStorageAccountName, sparkKeyVaultName); + Assert.AreEqual(expectedValue, actualValue, "Mount code is incorrect"); + } + + [TestMethod] + public void TestCreateLoadFunctionCode() + { + string realPath = "wasbs://mycontainer@teststorage.blob.core.windows.net/sample/udfsample.jar"; + string functionId = "myFunction"; + + // Test CreateLoadFunctionCode for UDF on HDInsight + string sparkType = "hdinsight"; + string functionType = "UDF"; + string actualValue = KernelService.CreateLoadFunctionCode( + realPath, + sparkType, + functionType, + functionId, + new Common.Models.PropertiesUD { ClassName = "SampleClass", Libs = new System.Collections.Generic.List(), Path = "keyvault://testkeyvault/sample" }); + string expectedValue = "val jarPath = \"wasbs://mycontainer@teststorage.blob.core.windows.net/sample/udfsample.jar\"\nval mainClass = \"SampleClass\"\nval jarFileUrl = datax.host.SparkJarLoader.addJarOnDriver(spark, jarPath, 0, false)\nspark.sparkContext.addJar(jarFileUrl)\ndatax.host.SparkJarLoader.registerJavaUDF(spark.udf, \"myFunction\", mainClass, null)\nprintln(\"done\")"; + Assert.AreEqual(expectedValue, actualValue, "Load UDF function code for HDInsight is incorrect"); + + // Test CreateLoadFunctionCode for UDF on Databricks + sparkType = "databricks"; + functionType = "UDF"; + actualValue = KernelService.CreateLoadFunctionCode( + realPath, + sparkType, + functionType, + functionId, + new Common.Models.PropertiesUD { ClassName = "SampleClass", Libs = new System.Collections.Generic.List(), Path = "keyvault://testkeyvault/sample" }); + expectedValue = "val jarPath = \"wasbs://mycontainer@teststorage.blob.core.windows.net/sample/udfsample.jar\"\nval mainClass = \"SampleClass\"\nval jarFileUrl = datax.host.SparkJarLoader.addJarOnDriver(spark, jarPath, 0, true)\nspark.sparkContext.addJar(jarFileUrl)\ndatax.host.SparkJarLoader.registerJavaUDF(spark.udf, \"myFunction\", mainClass, null)\nprintln(\"done\")"; + Assert.AreEqual(expectedValue, actualValue, "Load UDF function code for Databricks is incorrect"); + + // Test CreateLoadFunctionCode for UDAF on HDInsight + sparkType = "hdinsight"; + functionType = "UDAF"; + actualValue = KernelService.CreateLoadFunctionCode( + realPath, + sparkType, + functionType, + functionId, + new Common.Models.PropertiesUD { ClassName = "SampleClass", Libs = new System.Collections.Generic.List(), Path = "keyvault://testkeyvault/sample" }); + expectedValue = "val jarPath = \"wasbs://mycontainer@teststorage.blob.core.windows.net/sample/udfsample.jar\"\nval mainClass = \"SampleClass\"\nval jarFileUrl = datax.host.SparkJarLoader.addJarOnDriver(spark, jarPath, 0, false)\nspark.sparkContext.addJar(jarFileUrl)\ndatax.host.SparkJarLoader.registerJavaUDAF(spark.udf, \"myFunction\", mainClass)\nprintln(\"done\")"; + Assert.AreEqual(expectedValue, actualValue, "Load UDAF function code for HDInsight is incorrect"); + + // Test CreateLoadFunctionCode for UDAF on Databricks + sparkType = "databricks"; + functionType = "UDAF"; + actualValue = KernelService.CreateLoadFunctionCode( + realPath, + sparkType, + functionType, + functionId, + new Common.Models.PropertiesUD { ClassName = "SampleClass", Libs = new System.Collections.Generic.List(), Path = "keyvault://testkeyvault/sample" }); + expectedValue = "val jarPath = \"wasbs://mycontainer@teststorage.blob.core.windows.net/sample/udfsample.jar\"\nval mainClass = \"SampleClass\"\nval jarFileUrl = datax.host.SparkJarLoader.addJarOnDriver(spark, jarPath, 0, true)\nspark.sparkContext.addJar(jarFileUrl)\ndatax.host.SparkJarLoader.registerJavaUDAF(spark.udf, \"myFunction\", mainClass)\nprintln(\"done\")"; + Assert.AreEqual(expectedValue, actualValue, "Load UDAF function code for Databricks is incorrect"); + } + } +} diff --git a/Services/DataX.Flow/DataX.Flow.InteractiveQuery/DataX.Flow.InteractiveQuery.csproj b/Services/DataX.Flow/DataX.Flow.InteractiveQuery/DataX.Flow.InteractiveQuery.csproj index 3c907c3b2..a4ba0d604 100644 --- a/Services/DataX.Flow/DataX.Flow.InteractiveQuery/DataX.Flow.InteractiveQuery.csproj +++ b/Services/DataX.Flow/DataX.Flow.InteractiveQuery/DataX.Flow.InteractiveQuery.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -25,7 +25,7 @@ all runtime; build; native; contentfiles; analyzers - + all runtime; build; native; contentfiles; analyzers @@ -43,7 +43,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/DataX.Flow.InteractiveQuery/Databricks/DatabricksKernel.cs b/Services/DataX.Flow/DataX.Flow.InteractiveQuery/Databricks/DatabricksKernel.cs index db7285bbe..9cf6fd182 100644 --- a/Services/DataX.Flow/DataX.Flow.InteractiveQuery/Databricks/DatabricksKernel.cs +++ b/Services/DataX.Flow/DataX.Flow.InteractiveQuery/Databricks/DatabricksKernel.cs @@ -3,6 +3,7 @@ // Licensed under the MIT License // ********************************************************************* using Newtonsoft.Json; +using Newtonsoft.Json.Linq; using System; using System.Collections.Generic; using System.Net.Http; @@ -58,6 +59,21 @@ public string ExecuteCode(string code) var request = new HttpRequestMessage(HttpMethod.Post, url) { Content = new FormUrlEncodedContent(nvc) }; var response = client.SendAsync(request).Result; var responseString = response.Content.ReadAsStringAsync().Result; + if (!response.IsSuccessStatusCode) + { + //If request fails then return error string + try + { + var responseObj = JObject.Parse(responseString); + var errorMsg = responseObj["error"]; + return $"{response.ReasonPhrase}: {errorMsg?.ToString()}"; + } + catch + { + return $"{response.ReasonPhrase}"; + } + + } string commandId = JsonConvert.DeserializeObject(responseString).Id; // Now get the result output @@ -69,12 +85,11 @@ public string ExecuteCode(string code) var responsestring2 = response2.Content.ReadAsStringAsync().Result; result = JsonConvert.DeserializeObject(responsestring2); status = result.Status; - } while (status == "Running"); + } while (status != "Finished"); client.Dispose(); - return JsonConvert.SerializeObject(result.Results); + return (result.Results.Data != null) ? result.Results.Data.ToString() : ""; } - } /// @@ -94,6 +109,9 @@ public class Results [JsonProperty("resultType")] public string ResultType { get; set; } + [JsonProperty("data")] + public object Data { get; set; } + [JsonProperty("summary")] public object Summary { get; set; } diff --git a/Services/DataX.Flow/DataX.Flow.InteractiveQuery/Databricks/DatabricksKernelService.cs b/Services/DataX.Flow/DataX.Flow.InteractiveQuery/Databricks/DatabricksKernelService.cs index beff7e2bc..c636a4214 100644 --- a/Services/DataX.Flow/DataX.Flow.InteractiveQuery/Databricks/DatabricksKernelService.cs +++ b/Services/DataX.Flow/DataX.Flow.InteractiveQuery/Databricks/DatabricksKernelService.cs @@ -32,12 +32,13 @@ public class DatabricksKernelService: KernelService /// Config of the flow /// Spark connection info /// Logger for results/errors - public DatabricksKernelService(FlowConfigObject flowConfig, SparkConnectionInfo connectionInfo, ILogger logger) : base(flowConfig, connectionInfo, logger) + /// databricks token + public DatabricksKernelService(FlowConfigObject flowConfig, SparkConnectionInfo connectionInfo, ILogger logger, string databricksToken) : base(flowConfig, connectionInfo, logger) { string region = string.IsNullOrEmpty(flowConfig.SparkRegion) ? flowConfig.ResourceGroupLocation : flowConfig.SparkRegion; _baseUrl = _baseUrl.Replace("$region", region); _clusterName = flowConfig.SparkClusterName; - _token = flowConfig.SparkUserToken; + _token = databricksToken; } /// @@ -81,6 +82,7 @@ public override async Task DeleteKernelAsync(string kernelId) try { + InitializeClusterId(); // Set body string body = "{\"contextId\":\""+kernelId+"\", \"clusterId\":\""+_clusterId+"\"}"; var content = new StringContent(body); @@ -139,6 +141,8 @@ private void InitializeClusterId() if (cluster.Name == _clusterName) { _clusterId = cluster.Id; + client.Dispose(); + return; } } client.Dispose(); diff --git a/Services/DataX.Flow/DataX.Flow.InteractiveQuery/InteractiveQueryManager.cs b/Services/DataX.Flow/DataX.Flow.InteractiveQuery/InteractiveQueryManager.cs index 1b83b9758..29e90b8e1 100644 --- a/Services/DataX.Flow/DataX.Flow.InteractiveQuery/InteractiveQueryManager.cs +++ b/Services/DataX.Flow/DataX.Flow.InteractiveQuery/InteractiveQueryManager.cs @@ -12,7 +12,9 @@ using System.Collections.Generic; using System.IO; using System.Threading.Tasks; - +using Microsoft.Extensions.Configuration; +using System.Text.RegularExpressions; + namespace DataX.Flow.InteractiveQuery { /// @@ -22,19 +24,22 @@ public class InteractiveQueryManager { private string _flowContainerName => _engineEnvironment.EngineFlowConfig.FlowContainerName; private const string _GarbageCollectBlobName = "kernelList.json"; - private EngineEnvironment _engineEnvironment = new EngineEnvironment(); + private EngineEnvironment _engineEnvironment; private readonly ILogger _logger; - private const string _HDInsight = "HDInsight"; - private const string _DataBricks = "DataBricks"; + private const string _HDInsight = Config.ConfigDataModel.Constants.SparkTypeHDInsight; + private const string _DataBricks = Config.ConfigDataModel.Constants.SparkTypeDataBricks; private readonly string _sparkType = _HDInsight; + private readonly IConfiguration _configuration; - public InteractiveQueryManager(ILogger logger) + public InteractiveQueryManager(ILogger logger, IConfiguration configuration) { _logger = logger; + _configuration = configuration; + _engineEnvironment = new EngineEnvironment(_configuration); } - private string SetupSteps { get; set; } = string.Empty; - + private string SetupSteps { get; set; } = string.Empty; + /// /// This method gets called for api/kernel /// @@ -61,8 +66,10 @@ public async Task CreateAndInitializeKernel(JObject jObject) } var hashValue = Helper.GetHashCode(diag.UserName); - string sampleDataPath = Path.Combine(_engineEnvironment.OpsSparkSamplePath, $"{diag.Name}-{hashValue}.json"); - + string sampleDataPath = Helper.SetValueBasedOnSparkType(_engineEnvironment.EngineFlowConfig.SparkType, + Path.Combine(_engineEnvironment.OpsSparkSamplePath, $"{diag.Name}-{hashValue}.json"), + Helper.ConvertToDbfsFilePath(_engineEnvironment.OpsSparkSamplePath, $"{diag.Name}-{hashValue}.json")); + response = await CreateAndInitializeKernelHelper(diag.InputSchema, diag.UserName, diag.Name, sampleDataPath, diag.NormalizationSnippet, diag.ReferenceDatas, diag.Functions); if (response.Error.HasValue && response.Error.Value) { @@ -115,8 +122,9 @@ public async Task GetSampleInputFromQuery(JObject queryObject) /// /// The list of kernels to be deleted /// subscriptionId as passed in from the frontend + /// flow name /// Returns success or error message as the case maybe - public async Task DeleteKernelList(List kernels) + public async Task DeleteKernelList(List kernels, string flowName) { var response = await _engineEnvironment.GetEnvironmentVariables().ConfigureAwait(false); var subscriptionId = Helper.GetSecretFromKeyvaultIfNeeded(_engineEnvironment.EngineFlowConfig.SubscriptionId); @@ -128,7 +136,7 @@ public async Task DeleteKernelList(List kernels) foreach (var k in kernels) { - await DeleteKernelHelper(subscriptionId, k).ConfigureAwait(false); + await DeleteKernelHelper(subscriptionId, k, flowName).ConfigureAwait(false); } return ApiResult.CreateSuccess("success"); @@ -139,8 +147,9 @@ public async Task DeleteKernelList(List kernels) /// /// subscriptionId as passed in from the frontend /// kernelId as passed in from the frontend + /// flowName as passed in from the frontend /// Returns success or failure as the case may be - public async Task DeleteKernel(string kernelId) + public async Task DeleteKernel(string kernelId, string flowName) { var response = await _engineEnvironment.GetEnvironmentVariables().ConfigureAwait(false); var subscriptionId = Helper.GetSecretFromKeyvaultIfNeeded(_engineEnvironment.EngineFlowConfig.SubscriptionId); @@ -150,7 +159,7 @@ public async Task DeleteKernel(string kernelId) return ApiResult.CreateError(response.Message); } - response = await DeleteKernelHelper(subscriptionId, kernelId).ConfigureAwait(false); + response = await DeleteKernelHelper(subscriptionId, kernelId, flowName).ConfigureAwait(false); if (response.Error.HasValue && response.Error.Value) { _logger.LogError(response.Message); @@ -207,11 +216,14 @@ public async Task RecycleKernelHelper(InteractiveQueryObject diag, bo diag.Name = await _engineEnvironment.GetUniqueName(Helper.GetSecretFromKeyvaultIfNeeded(_engineEnvironment.EngineFlowConfig.SubscriptionId), diag.DisplayName); } - KernelService kernelService = CreateKernelService(); + KernelService kernelService = CreateKernelService(diag.Name); //Create the xml with the scala steps to execute to initialize the kernel - var hashValue = Helper.GetHashCode(diag.UserName); - var sampleDataPath = Path.Combine(_engineEnvironment.OpsSparkSamplePath, $"{diag.Name}-{hashValue}.json"); + var hashValue = Helper.GetHashCode(diag.UserName); + var sampleDataPath = Helper.SetValueBasedOnSparkType(_engineEnvironment.EngineFlowConfig.SparkType, + Path.Combine(_engineEnvironment.OpsSparkSamplePath, $"{diag.Name}-{hashValue}.json"), + Helper.ConvertToDbfsFilePath(_engineEnvironment.OpsSparkSamplePath, $"{diag.Name}-{hashValue}.json")); + DiagnosticInputhelper(diag.InputSchema, sampleDataPath, diag.NormalizationSnippet, diag.Name); response = await kernelService.RecycleKernelAsync(diag.KernelId, diag.UserName, diag.Name, _engineEnvironment.OpsBlobConnectionString, Path.Combine(_engineEnvironment.OpsDiagnosticPath, _GarbageCollectBlobName), SetupSteps, isReSample, diag.ReferenceDatas, diag.Functions); @@ -225,11 +237,12 @@ public async Task RecycleKernelHelper(InteractiveQueryObject diag, bo } } - private KernelService CreateKernelService() + private KernelService CreateKernelService(string flowName) { if (_engineEnvironment.EngineFlowConfig.SparkType == _DataBricks) { - return new Databricks.DatabricksKernelService(_engineEnvironment.EngineFlowConfig, _engineEnvironment.SparkConnInfo, _logger); + var databricksTokenSecret = $"secretscope://{_engineEnvironment.EngineFlowConfig.SparkKeyVaultName}/{flowName}-info-databricksToken"; + return new Databricks.DatabricksKernelService(_engineEnvironment.EngineFlowConfig, _engineEnvironment.SparkConnInfo, _logger, Helper.GetSecretFromKeyvaultIfNeeded(databricksTokenSecret)); } else { @@ -240,9 +253,9 @@ private KernelService CreateKernelService() /// /// This is the API method that gets called from the front end on a regular cadence and will delete all the kernels that are more than 3 hours old /// - /// SubscriptionId + /// flowName /// Returns the result whether the list of kernels were deleted or not. We don't fail if one of the kernels fails to delete because it just does not exist - public async Task DeleteKernels() + public async Task DeleteKernels(string flowName) { var response = await _engineEnvironment.GetEnvironmentVariables().ConfigureAwait(false); var subscriptionId = Helper.GetSecretFromKeyvaultIfNeeded(_engineEnvironment.EngineFlowConfig.SubscriptionId); @@ -251,7 +264,7 @@ public async Task DeleteKernels() _logger.LogError(response.Message); return ApiResult.CreateError(response.Message); } - KernelService kernelService = CreateKernelService(); + KernelService kernelService = CreateKernelService(flowName); response = await kernelService.GarbageCollectListOfKernels(_engineEnvironment.OpsBlobConnectionString, Path.Combine(_engineEnvironment.OpsDiagnosticPath, _GarbageCollectBlobName)).ConfigureAwait(false); if (response.Error.HasValue && response.Error.Value) @@ -266,17 +279,17 @@ public async Task DeleteKernels() /// /// This is the API method that gets called from the front end on a regular cadence and will delete all the kernels that were created /// - /// jObject + /// flowName /// Returns the result whether the list of kernels were deleted or not. We don't fail if one of the kernels fails to delete because it just does not exist - public async Task DeleteAllKernels() - { + public async Task DeleteAllKernels(string flowName) + { var response = await _engineEnvironment.GetEnvironmentVariables().ConfigureAwait(false); if (response.Error.HasValue && response.Error.Value) { _logger.LogError(response.Message); return ApiResult.CreateError(response.Message); } - KernelService kernelService = CreateKernelService(); + KernelService kernelService = CreateKernelService(flowName); _logger.LogInformation("Deleting all Kernels..."); response = await kernelService.GarbageCollectListOfKernels(_engineEnvironment.OpsBlobConnectionString, Path.Combine(_engineEnvironment.OpsDiagnosticPath, _GarbageCollectBlobName), true).ConfigureAwait(false); @@ -295,7 +308,7 @@ public async Task DeleteAllKernels() /// /// rawSchema as passed in from the frontend /// userId as passed in from the frontend - /// rawSchema as passed in from the frontend + /// flowId as passed in from the frontend /// sampleDataPath where the sample data is stored /// normalizationSnippet as passed in from the frontend /// referenceDatas as passed in from the frontend @@ -308,7 +321,7 @@ private async Task CreateAndInitializeKernelHelper(string rawSchema, //Create the xml with the scala steps to execute to initialize the kernel DiagnosticInputhelper(rawSchema, sampleDataPath, normalizationSnippet, flowId); - KernelService kernelService = CreateKernelService(); + KernelService kernelService = CreateKernelService(flowId); var response = await kernelService.GarbageCollectListOfKernels(_engineEnvironment.OpsBlobConnectionString, Path.Combine(_engineEnvironment.OpsDiagnosticPath, _GarbageCollectBlobName)); response = await kernelService.CreateKernelAsync(); @@ -325,6 +338,11 @@ private async Task CreateAndInitializeKernelHelper(string rawSchema, return ApiResult.CreateError(response.Message); } + if(_engineEnvironment.EngineFlowConfig.SparkType == Config.ConfigDataModel.Constants.SparkTypeDataBricks) + { + kernelService.MountStorage(_engineEnvironment.EngineFlowConfig.OpsStorageAccountName, _engineEnvironment.EngineFlowConfig.SparkKeyVaultName, sampleDataPath, kernelId); + } + response = await kernelService.CreateandInitializeKernelAsync(kernelId, SetupSteps, false, referenceDatas, functions); if (response.Error.HasValue && response.Error.Value) { @@ -373,7 +391,7 @@ private async Task ExecuteQueryHelper(JObject jObject) try { - KernelService kernelService = CreateKernelService(); + KernelService kernelService = CreateKernelService(query.Name); var result = await kernelService.ExecuteQueryAsync(query.Query, query.KernelId); return result; } @@ -401,7 +419,7 @@ private async Task GetSampleInputFromQueryHelper(JObject jObject) var query = jObject.ToObject(); try { - KernelService kernelService = CreateKernelService(); + KernelService kernelService = CreateKernelService(query.Name); var result = await kernelService.GetSampleInputFromQueryAsync(query.Query, query.KernelId); return result; } @@ -417,8 +435,9 @@ private async Task GetSampleInputFromQueryHelper(JObject jObject) /// /// subscriptionId that is passed in from the frontend /// kernelId that needs to be deleted + /// flowName /// Returns success or failure after the delete kernel api is called - private async Task DeleteKernelHelper(string subscriptionId, string kernelId) + private async Task DeleteKernelHelper(string subscriptionId, string kernelId, string flowName) { // validate KernelId can't be null @@ -430,7 +449,7 @@ private async Task DeleteKernelHelper(string subscriptionId, string k try { - KernelService kernelService = CreateKernelService(); + KernelService kernelService = CreateKernelService(flowName); var result = await kernelService.DeleteKernelAsync(kernelId); if (!(result.Error.HasValue && result.Error.Value)) { @@ -475,7 +494,7 @@ private void DiagnosticInputhelper(string rawSchema, string sampleDataPath, stri { ["RawSchema"] = rawSchema, ["SampleDataPath"] = sampleDataPath, - ["NormalizationSnippet"] = finalNormalizationString, + ["NormalizationSnippet"] = finalNormalizationString, ["BinName"] = TranslateBinNames(_engineEnvironment.EngineFlowConfig.BinaryName, _engineEnvironment.EngineFlowConfig.OpsStorageAccountName, _engineEnvironment.EngineFlowConfig.InteractiveQueryDefaultContainer), ["KernelDisplayName"] = _engineEnvironment.GenerateKernelDisplayName(flowId) }; diff --git a/Services/DataX.Flow/DataX.Flow.InteractiveQuery/KernelService.cs b/Services/DataX.Flow/DataX.Flow.InteractiveQuery/KernelService.cs index e650fed6e..6dfbd59a8 100644 --- a/Services/DataX.Flow/DataX.Flow.InteractiveQuery/KernelService.cs +++ b/Services/DataX.Flow/DataX.Flow.InteractiveQuery/KernelService.cs @@ -23,10 +23,13 @@ namespace DataX.Flow.InteractiveQuery { - public abstract class KernelService + /// + /// This is the class that manages the kernel for interactive queries + /// + public abstract class KernelService { private steps _steps = null; - private const int _MaxCount = 20; + private const int _MaxCount = 20; private string _setupStepsXml = ""; private const string _QuerySeparator = "--DataXQuery--"; @@ -41,6 +44,7 @@ public KernelService(FlowConfigObject flowConfig, SparkConnectionInfo connection { SparkType = flowConfig.SparkType; Logger = logger; + FlowConfig = flowConfig; } protected FlowConfigObject FlowConfig { get; set; } @@ -62,7 +66,7 @@ public KernelService(FlowConfigObject flowConfig, SparkConnectionInfo connection /// All UDFs and UDAFs as specified by the user /// ApiResult which contains error or the kernelId (along with/without warning) as the case maybe public async Task CreateandInitializeKernelAsync(string kernelId, string setupStepsXml, bool isReSample, List referenceDatas, List functions) - { + { _setupStepsXml = setupStepsXml; var response = await InitializeKernelAsync(kernelId, isReSample, referenceDatas, functions); if (response.Error.HasValue && response.Error.Value) @@ -72,7 +76,7 @@ public async Task CreateandInitializeKernelAsync(string kernelId, str return ApiResult.CreateError(response.Message); } else - { + { return ApiResult.CreateSuccess(response.Result); } } @@ -250,7 +254,7 @@ private ApiResult LoadandRunSteps(IKernel kernel, bool isReSample, List referen for (int i = 0; i < referenceDatas.Count; i++) { - string realPath = UDFPathResolver(referenceDatas[i].Properties.Path); + string realPath = Helper.SetValueBasedOnSparkType(SparkType, + UDFPathResolver(referenceDatas[i].Properties.Path), + Helper.ConvertToDbfsFilePath(UDFPathResolver(referenceDatas[i].Properties.Path))); + + if (SparkType == DataX.Config.ConfigDataModel.Constants.SparkTypeDataBricks) + { + MountStorage(FlowConfig.OpsStorageAccountName, FlowConfig.SparkKeyVaultName, realPath, kernel); + } + string code = $"spark.read.option(\"delimiter\", \"{(string.IsNullOrEmpty(referenceDatas[i].Properties.Delimiter) ? "," : referenceDatas[i].Properties.Delimiter)}\").option(\"header\", \"{referenceDatas[i].Properties.Header.ToString()}\").csv(\"{realPath}\").createOrReplaceTempView(\"{referenceDatas[i].Id}\"); print(\"done\");"; string result = kernel.ExecuteCode(code); LogErrors(result, code, "LoadReferenceData"); @@ -317,56 +329,71 @@ private void LoadFunctions(IKernel kernel, List functions) { return; } - foreach (FunctionObject fo in functions) { - if(fo.TypeDisplay=="UDF") - { - PropertiesUD properties = JsonConvert.DeserializeObject(fo.Properties.ToString()); - string realPath = UDFPathResolver(properties.Path); - string code = $"val jarPath = \"{realPath}\"\n"; - code += $"val mainClass = \"{properties.ClassName}\"\n"; - code += $"datax.host.SparkJarLoader.addJarOnDriver(spark, jarPath, 0, false)\n"; - code += "spark.sparkContext.addJar(jarPath)\n"; - code += $"datax.host.SparkJarLoader.registerJavaUDF(spark.udf, \"{fo.Id}\", mainClass, null)\n"; - - if(properties.Libs!=null) - { - foreach (string lib in properties.Libs) - { - code += $"datax.host.SparkJarLoader.addJar(spark, \"{lib}\")\n"; - } - } - code += "println(\"done\")"; - string result = kernel.ExecuteCode(code); - LogErrors(result, code, "LoadFunctions"); + switch (fo.TypeDisplay) + { + case "UDF": + case "UDAF": + PropertiesUD properties = JsonConvert.DeserializeObject(fo.Properties.ToString()); + string realPath = UDFPathResolver(properties.Path); + string code = CreateLoadFunctionCode(realPath, SparkType, fo.TypeDisplay, fo.Id, properties); + string result = kernel.ExecuteCode(code); + LogErrors(result, code, "LoadFunctions"); + break; + case "Azure Function": + // TODO + break; + default: + break; } - else if(fo.TypeDisplay=="UDAF") - { - PropertiesUD properties = JsonConvert.DeserializeObject(fo.Properties.ToString()); - string realPath = UDFPathResolver(properties.Path); - string code = $"val jarPath = \"{realPath}\"\n"; - code += $"val mainClass = \"{properties.ClassName}\"\n"; - code += $"datax.host.SparkJarLoader.addJarOnDriver(spark, jarPath, 0, false)\n"; - code += "spark.sparkContext.addJar(jarPath)\n"; - code += $"datax.host.SparkJarLoader.registerJavaUDAF(spark.udf, \"{fo.Id}\", mainClass)\n"; + } + } - if (properties.Libs != null) - { - foreach (string lib in properties.Libs) - { - code += $"datax.host.SparkJarLoader.addJar(spark, \"{lib}\")\n"; - } - } - code += "println(\"done\")"; - string result = kernel.ExecuteCode(code); - LogErrors(result, code, "LoadFunctions"); - } - else if(fo.TypeDisplay=="Azure Function") + /// + /// Create code to load functions + /// + /// Path of jar + /// spark type + /// function type + /// functionId + /// Function properties + /// code to load functions + public static string CreateLoadFunctionCode(string realPath, string sparkType, string functionType, string functionId, PropertiesUD properties) + { + string code = $"val jarPath = \"{realPath}\"\n"; + code += $"val mainClass = \"{properties.ClassName}\"\n"; + + if (sparkType == DataX.Config.ConfigDataModel.Constants.SparkTypeDataBricks) + { + code += $"val jarFileUrl = datax.host.SparkJarLoader.addJarOnDriver(spark, jarPath, 0, true)\n"; + } + else + { + code += $"val jarFileUrl = datax.host.SparkJarLoader.addJarOnDriver(spark, jarPath, 0, false)\n"; + } + + code += "spark.sparkContext.addJar(jarFileUrl)\n"; + + if (functionType == "UDF") + { + code += $"datax.host.SparkJarLoader.registerJavaUDF(spark.udf, \"{functionId}\", mainClass, null)\n"; + } + else if (functionType == "UDAF") + { + code += $"datax.host.SparkJarLoader.registerJavaUDAF(spark.udf, \"{functionId}\", mainClass)\n"; + } + + if (properties.Libs != null) + { + foreach (string lib in properties.Libs) { - // TODO + code += $"datax.host.SparkJarLoader.addJar(spark, \"{lib}\")\n"; } } + code += "println(\"done\")"; + + return code; } /// @@ -577,7 +604,7 @@ public async Task RecycleKernelAsync(string kernelId, string userId, var deleteResult = await DeleteKernelAsync(kernelId); } var response = await GarbageCollectListOfKernels(connectionString, blobUri); - + response = await CreateKernelAsync(); if (response.Error.HasValue && response.Error.Value) { @@ -664,7 +691,7 @@ private void LogErrors(string result, string code, string step) Logger.LogError($"Initialization step - {step}: Resulting Error: {errors}"); } } - + /// /// Returning a json object /// @@ -675,7 +702,7 @@ private ApiResult ConvertToJson(int maxCount, string result) { var error = CheckErrors(result); if (!string.IsNullOrEmpty(error)) - { + { return ApiResult.CreateError("{\"Error\": \"" + error + "\"}"); } else @@ -730,8 +757,8 @@ private string ReplaceNewLineFeed(string input) private string UDFPathResolver(string path) { if (path != null && Config.Utility.KeyVaultUri.IsSecretUri(path)) - { - Regex r = new Regex(@"^((keyvault:?):\/\/)?([^:\/\s]+)(\/)(.*)?", RegexOptions.IgnoreCase); + { + Regex r = new Regex(@"^((keyvault|secretscope:?):\/\/)?([^:\/\s]+)(\/)(.*)?", RegexOptions.IgnoreCase); var keyvault = string.Empty; var secret = string.Empty; @@ -751,6 +778,67 @@ private string UDFPathResolver(string path) return path; } + /// + /// Mount container to DBFS + /// + /// Storage account name + /// Spark keyvault name + /// DBFS path. Format dbfs:/mnt/livequery/.. + /// kernelId + public void MountStorage(string opsStorageAccountName, string sparkKeyVaultName, string dbfsPath, string kernelId) + { + // Attach to kernel + IKernel kernel = GetKernel(kernelId); + MountStorage(opsStorageAccountName, sparkKeyVaultName, dbfsPath, kernel); + } + + /// + /// Mount container to DBFS + /// + /// Storage account name + /// Spark keyvault name + /// DBFS path. Format dbfs:/mnt/livequery/.. + /// kernel + private void MountStorage(string opsStorageAccountName, string sparkKeyVaultName, string dbfsPath, IKernel kernel) + { + try + { + //Verify the container has been mounted to DBFS + string verifyCode = $"dbutils.fs.ls(\"{dbfsPath}\")"; + var result = kernel.ExecuteCode(verifyCode); + + //If container is not mounted then mount it + if (string.IsNullOrEmpty(result)) + { + string mountCode = CreateMountCode(dbfsPath, opsStorageAccountName, sparkKeyVaultName); + kernel.ExecuteCode(mountCode); + } + } + catch (Exception ex) + { + Logger.LogError(ex, ex.Message); + } + } + + /// + /// Create mount command to execute + /// + /// DBFS path. Format dbfs:/mnt/livequery/.. + /// Ops Storage Account Name + /// Spark KeyVault Name + /// Returns the mount code + public static string CreateMountCode(string dbfsPath, string opsStorageAccountName, string sparkKeyVaultName) + { + Regex r = new Regex($"{Config.ConfigDataModel.Constants.PrefixDbfs}{Config.ConfigDataModel.Constants.PrefixDbfsMount}([a-zA-Z0-9-]*)/", RegexOptions.IgnoreCase); + string containerName = r.Match(dbfsPath).Groups[1].Value; + string mountCode = $"dbutils.fs.mount(" + + $"source = \"wasbs://{containerName}@{opsStorageAccountName}.blob.core.windows.net/\", " + + $"mountPoint = \"/{Config.ConfigDataModel.Constants.PrefixDbfsMount}/{containerName}\", " + + $"extraConfigs = Map(" + + $"\"fs.azure.account.key.{opsStorageAccountName}.blob.core.windows.net\"->" + + $"dbutils.secrets.get(scope = \"{sparkKeyVaultName}\", key = \"{Config.ConfigDataModel.Constants.AccountSecretPrefix}{opsStorageAccountName}\")))"; + return mountCode; + } } /// /// Kernel Properties that are entered in the blob for garbage collection diff --git a/Services/DataX.Flow/DataX.Flow.Scheduler/DataX.Flow.Scheduler.csproj b/Services/DataX.Flow/DataX.Flow.Scheduler/DataX.Flow.Scheduler.csproj new file mode 100644 index 000000000..4271fc243 --- /dev/null +++ b/Services/DataX.Flow/DataX.Flow.Scheduler/DataX.Flow.Scheduler.csproj @@ -0,0 +1,36 @@ + + + + netcoreapp2.2 + true + FinalPublicKey.snk + true + + + + + all + runtime; build; native; contentfiles; analyzers + + + + all + runtime; build; native; contentfiles; analyzers + + + + + + + + + + + + + Microsoft400 + StrongName + + + + diff --git a/Services/DataX.Flow/DataX.Flow.Scheduler/FinalPublicKey.snk b/Services/DataX.Flow/DataX.Flow.Scheduler/FinalPublicKey.snk new file mode 100644 index 000000000..110b59c7b Binary files /dev/null and b/Services/DataX.Flow/DataX.Flow.Scheduler/FinalPublicKey.snk differ diff --git a/Services/DataX.Flow/DataX.Flow.Scheduler/TimedScheduler.cs b/Services/DataX.Flow/DataX.Flow.Scheduler/TimedScheduler.cs new file mode 100644 index 000000000..fdbc70d39 --- /dev/null +++ b/Services/DataX.Flow/DataX.Flow.Scheduler/TimedScheduler.cs @@ -0,0 +1,119 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Flow.Common; +using DataX.Gateway.Contract; +using DataX.Utility.ServiceCommunication; +using Microsoft.Extensions.Configuration; +using Microsoft.Extensions.Hosting; +using Microsoft.Extensions.Logging; +using Microsoft.IdentityModel.Clients.ActiveDirectory; +using System; +using System.Collections.Generic; +using System.IdentityModel.Tokens.Jwt; +using System.Linq; +using System.Net.Http; +using System.Threading; +using System.Threading.Tasks; + +namespace DataX.Flow.Scheduler +{ + public class TimedScheduler : BackgroundService + { + private readonly ILogger _logger; + private readonly IConfiguration _configuration; + private readonly EngineEnvironment _engineEnvironment; + + // Frequency at which to run the scheduler + private readonly int _schedulerWakeupFrequencyInMin = 60; + private readonly int _oneMinInMilliSeconds = 60 * 1000; + + internal static InterServiceCommunicator Communicator + { + private get; + set; + } + + public TimedScheduler(ILogger logger, IConfiguration configuration) + { + _logger = logger; + _configuration = configuration; + _engineEnvironment = new EngineEnvironment(_configuration); + + Communicator = new InterServiceCommunicator(new TimeSpan(0, 4, 0)); + } + + protected override async Task ExecuteAsync(CancellationToken stoppingToken) + { + _logger.LogInformation($"{DateTime.UtcNow}:TimedScheduler is starting."); + + stoppingToken.Register(() => + _logger.LogInformation($"{DateTime.UtcNow}: TimedScheduler background task is stopping.")); + + while (!stoppingToken.IsCancellationRequested) + { + _logger.LogInformation($"{DateTime.UtcNow}:TimedScheduler task doing background work."); + + await StartBatchJobs().ConfigureAwait(false); + + await Task.Delay(_schedulerWakeupFrequencyInMin * _oneMinInMilliSeconds, stoppingToken).ConfigureAwait(false); + } + + _logger.LogInformation($"{DateTime.UtcNow}: TimedScheduler background task is stopping."); + } + + private async Task> SetRequestHeader() + { + var response = await _engineEnvironment.GetEnvironmentVariables().ConfigureAwait(false); + if (response.Error.HasValue && response.Error.Value) + { + _logger.LogError(response.Message); + throw new Exception("Can't get environment variables."); + } + var clientId = _engineEnvironment.EngineFlowConfig.ConfiggenClientId; + var clientSecret = Helper.GetSecretFromKeyvaultIfNeeded(_engineEnvironment.EngineFlowConfig.ConfiggenClientSecret); + var clientResourceId = Helper.GetSecretFromKeyvaultIfNeeded(_engineEnvironment.EngineFlowConfig.ConfiggenClientResourceId); + var tenantId = _engineEnvironment.EngineFlowConfig.ConfiggenTenantId; + + var authenticationContext = new AuthenticationContext($"https://login.windows.net/{tenantId}"); + var credential = new ClientCredential(clientId, clientSecret); + var apiToken = authenticationContext.AcquireTokenAsync(clientResourceId, credential).Result.AccessToken; + + var jwtHandler = new JwtSecurityTokenHandler(); + var readableToken = jwtHandler.CanReadToken(apiToken); + + Dictionary headers = new Dictionary(); + if (readableToken == true) + { + var token = jwtHandler.ReadJwtToken(apiToken); + + var roleValue = token.Claims.FirstOrDefault(c => c.Type == "roles")?.Value; + if (!string.IsNullOrEmpty(roleValue)) + { + headers.Add(Constants.UserRolesHeader, roleValue); + + } + } + + return headers; + } + + private async Task StartBatchJobs() + { + var headers = await SetRequestHeader().ConfigureAwait(false); + + _logger.LogInformation($"{DateTime.UtcNow}:TimedScheduler Starting batch jobs"); + + try + { + await Communicator.InvokeServiceAsync(HttpMethod.Post, "DataX.Flow", "Flow.ManagementService", "flow/schedulebatch", headers).ConfigureAwait(false); + } + catch (Exception e) + { + var message = e.InnerException == null ? e.Message : e.InnerException.Message; + _logger.LogInformation($"{DateTime.UtcNow}:TimedScheduler an exception is thrown:" + message); + } + } + } +} diff --git a/Services/DataX.Flow/DataX.Flow.SchemaInference.Tests/DataX.Flow.SchemaInference.Tests.csproj b/Services/DataX.Flow/DataX.Flow.SchemaInference.Tests/DataX.Flow.SchemaInference.Tests.csproj index b7a5bbeca..ddb1ab69a 100644 --- a/Services/DataX.Flow/DataX.Flow.SchemaInference.Tests/DataX.Flow.SchemaInference.Tests.csproj +++ b/Services/DataX.Flow/DataX.Flow.SchemaInference.Tests/DataX.Flow.SchemaInference.Tests.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 false @@ -67,7 +67,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/DataX.Flow.SchemaInference.Tests/SchemaUtilityTests.cs b/Services/DataX.Flow/DataX.Flow.SchemaInference.Tests/SchemaUtilityTests.cs new file mode 100644 index 000000000..e5c1f289c --- /dev/null +++ b/Services/DataX.Flow/DataX.Flow.SchemaInference.Tests/SchemaUtilityTests.cs @@ -0,0 +1,104 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Utilities.Blob; +using Microsoft.VisualStudio.TestTools.UnitTesting; +using Newtonsoft.Json; +using System; +using System.Collections.Generic; +using System.IO; + +namespace DataX.Flow.SchemaInference.Tests +{ + [TestClass] + public class SchemaUtilityTests + { + [TestMethod] + public void BlobPathPrefixTest() + { + List> blobPaths = new List> + { + new Tuple( + @"wasbs://mycontainer@mysa.blob.core.windows.net/mypath/mypath2/mypath3/{yyyy}/{MM}/{dd}", + "mycontainer", + @"mypath/mypath2/mypath3/", + @"mysa.blob.core.windows.net/mycontainer/mypath/mypath2/mypath3/(\w+)/(\w+)/(\w+)" + ), + new Tuple( + @"wasbs://mycontainer@mysa.blob.core.windows.net/mypath/mypath2/mypath3/2018/07/12/subfolder/00/test", + "mycontainer", + @"mypath/mypath2/mypath3/2018/07/12/subfolder/00/test", + @"mysa.blob.core.windows.net/mycontainer/mypath/mypath2/mypath3/2018/07/12/subfolder/00/test" + ), + new Tuple( + @"wasbs://myoutputs@somesa.blob.core.windows.net/Test/{yyyy-MM-dd}", + "myoutputs", + @"Test/", + @"somesa.blob.core.windows.net/myoutputs/Test/(\w+)-(\w+)-(\w+)" + ), + new Tuple( + @"wasbs://myoutputs@somesa.blob.core.windows.net/{yyyy-MM-dd}", + "myoutputs", + @"", + @"somesa.blob.core.windows.net/myoutputs/(\w+)-(\w+)-(\w+)" + ), + new Tuple( + @"wasbs://myoutputs@somesa.blob.core.windows.net/Test/{yyyy}/{MM}/{dd}", + "myoutputs", + @"Test/", + @"somesa.blob.core.windows.net/myoutputs/Test/(\w+)/(\w+)/(\w+)" + ), + new Tuple( + @"wasbs://myoutputs@somesa.blob.core.windows.net/{yyyy}/{MM}/{dd}", + "myoutputs", + @"", + @"somesa.blob.core.windows.net/myoutputs/(\w+)/(\w+)/(\w+)" + ), + new Tuple( + @"wasbs://myoutputs@somesa.blob.core.windows.net/{yyyy}/{MM}/{dd}/Test", + "myoutputs", + @"", + @"somesa.blob.core.windows.net/myoutputs/(\w+)/(\w+)/(\w+)/Test" + ), + new Tuple( + @"wasbs://myoutputs@somesa.blob.core.windows.net/{yyyy-MM-dd}/Test", + "myoutputs", + @"", + @"somesa.blob.core.windows.net/myoutputs/(\w+)-(\w+)-(\w+)/Test" + ), + new Tuple( + @"wasbs://mycontainer@mysa.blob.core.windows.net/mypath/mypath2/mypath3/{yyyy/MM/dd}", + "mycontainer", + @"mypath/mypath2/mypath3/", + @"mysa.blob.core.windows.net/mycontainer/mypath/mypath2/mypath3/(\w+)/(\w+)/(\w+)" + ), + new Tuple( + @"wasbs://mycontainer@mysa.blob.core.windows.net/mypath/mypath2/mypath3/{yyyy/MM/dd}/test", + "mycontainer", + @"mypath/mypath2/mypath3/", + @"mysa.blob.core.windows.net/mycontainer/mypath/mypath2/mypath3/(\w+)/(\w+)/(\w+)/test" + ) + }; + + foreach (var blobPath in blobPaths) + { + var wasbPath = blobPath.Item1; + if (!Uri.TryCreate(wasbPath, UriKind.Absolute, out var uri)) + { + Assert.Fail("blob path is incorrect"); + } + + var containerName = uri.UserInfo; + Assert.AreEqual(blobPath.Item2, containerName, "container name is incorrect"); + + var path = uri.Host + "/" + uri.UserInfo + uri.LocalPath; + var pathPattern = BlobHelper.GenerateRegexPatternFromPath(path); + Assert.AreEqual(blobPath.Item4, pathPattern, "pattern is incorrect"); + + var prefix = BlobHelper.ParsePrefix(wasbPath); + Assert.AreEqual(blobPath.Item3, prefix, "Prefix generation is incorrect"); + } + } + } +} diff --git a/Services/DataX.Flow/DataX.Flow.SchemaInference/Blob/BlobMessageBus.cs b/Services/DataX.Flow/DataX.Flow.SchemaInference/Blob/BlobMessageBus.cs new file mode 100644 index 000000000..8908fa0f1 --- /dev/null +++ b/Services/DataX.Flow/DataX.Flow.SchemaInference/Blob/BlobMessageBus.cs @@ -0,0 +1,77 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Config.ConfigDataModel; +using DataX.Contract.Exception; +using DataX.Flow.Common; +using DataX.Utilities.Blob; +using Microsoft.Extensions.Logging; +using Newtonsoft.Json; +using System; +using System.Collections.Generic; +using System.Text.RegularExpressions; +using System.Threading.Tasks; + +namespace DataX.Flow.SchemaInference.Blob +{ + public class BlobMessageBus : IMessageBus + { + private readonly List _batchInputs = null; + private readonly ILogger _logger; + + public BlobMessageBus(List batchInputs, ILogger logger) + { + _logger = logger; + _batchInputs = batchInputs; + } + + /// + /// Gets Sample Events + /// + /// seconds for which the sample data is fetched + /// Returns EventsData object + public async Task GetSampleEvents(int seconds) + { + EventsData eventsData = new EventsData(); + const int numberOfDocumentsToRead = 500; + + foreach (var batchInput in _batchInputs) + { + var connection = Helper.GetSecretFromKeyvaultIfNeeded(batchInput.Properties.Connection); + var wasbPath = Helper.GetSecretFromKeyvaultIfNeeded(batchInput.Properties.Path); + + if (!Uri.TryCreate(wasbPath, UriKind.Absolute, out var uri)) + { + throw new ArgumentException($"Malformed Uri for the blob path:'{wasbPath}'. The blob path should be a wasbs url. e.g. wasbs://mycontainer@myaccount.blob.core.windows.net/mypath"); + } + + var path = uri.Host + "/" + uri.UserInfo + uri.LocalPath; + var pathPattern = BlobHelper.GenerateRegexPatternFromPath(path); + var containerName = uri.UserInfo; + var prefix = BlobHelper.ParsePrefix(wasbPath); + + var contents = await BlobHelper.GetLastModifiedBlobContentsInBlobPath(connection, containerName, prefix, pathPattern, numberOfDocumentsToRead).ConfigureAwait(false); + + foreach (var content in contents) + { + // Get raw data + EventRaw er = new EventRaw + { + Raw = content, + Properties = new Dictionary() { { "Length", content.Length.ToString() } }, + SystemProperties = new Dictionary() { { "Length", content.Length.ToString() } } + }; + + er.Json = JsonConvert.SerializeObject(er); + + eventsData.EventsJson += er.Json + "\r\n"; + + eventsData.Events.Add(er); + } + } + + return eventsData; + } + } +} diff --git a/Services/DataX.Flow/DataX.Flow.SchemaInference/DataX.Flow.SchemaInference.csproj b/Services/DataX.Flow/DataX.Flow.SchemaInference/DataX.Flow.SchemaInference.csproj index c1d408d80..2953815fd 100644 --- a/Services/DataX.Flow/DataX.Flow.SchemaInference/DataX.Flow.SchemaInference.csproj +++ b/Services/DataX.Flow/DataX.Flow.SchemaInference/DataX.Flow.SchemaInference.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -35,6 +35,7 @@ + @@ -43,7 +44,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/DataX.Flow.SchemaInference/Kafka/KafkaMessageBus.cs b/Services/DataX.Flow/DataX.Flow.SchemaInference/Kafka/KafkaMessageBus.cs index 8ecd520de..fd2e50437 100644 --- a/Services/DataX.Flow/DataX.Flow.SchemaInference/Kafka/KafkaMessageBus.cs +++ b/Services/DataX.Flow/DataX.Flow.SchemaInference/Kafka/KafkaMessageBus.cs @@ -107,10 +107,9 @@ public async Task GetSampleEvents(int seconds) { _logger.LogInformation($"Closing consumer"); consumer.Close(); - return eventsData; + return await Task.FromResult(eventsData).ConfigureAwait(false); } - if (consumeResult.IsPartitionEOF) { @@ -125,7 +124,7 @@ public async Task GetSampleEvents(int seconds) { Raw = consumeResult.Value, Properties = new Dictionary() { { "HeadersCount", consumeResult.Headers.Count.ToString() } }, - }; + }; // Set properties (using the Headers) if (consumeResult.Headers != null && consumeResult.Headers.Count > 0) @@ -172,7 +171,7 @@ public async Task GetSampleEvents(int seconds) { _logger.LogInformation($"Closing consumer"); consumer.Close(); - return eventsData; + return await Task.FromResult(eventsData).ConfigureAwait(false); } } } diff --git a/Services/DataX.Flow/DataX.Flow.SchemaInference/SchemaGenerator.cs b/Services/DataX.Flow/DataX.Flow.SchemaInference/SchemaGenerator.cs index 5096c1b10..59fec9417 100644 --- a/Services/DataX.Flow/DataX.Flow.SchemaInference/SchemaGenerator.cs +++ b/Services/DataX.Flow/DataX.Flow.SchemaInference/SchemaGenerator.cs @@ -8,6 +8,7 @@ using System.Threading.Tasks; using DataX.Config.ConfigDataModel; using DataX.Flow.Common; +using DataX.Flow.SchemaInference.Blob; using DataX.Flow.SchemaInference.Eventhub; using DataX.Flow.SchemaInference.Kafka; using DataX.Utilities.Blob; @@ -35,21 +36,28 @@ public class SchemaGenerator /// Destination for where samples are stored /// Type of MessageBus /// logger for Telemetry - public SchemaGenerator(string brokerList, string connectionString, List hubNames, string consumerGroup, string storageConnectionString, string checkpointContainerName, string blobDirectory, string inputType, ILogger logger) + public SchemaGenerator(string brokerList, string connectionString, List hubNames, string consumerGroup, string storageConnectionString, string checkpointContainerName, string blobDirectory, string inputType, string inputMode, List batchInputs, ILogger logger) { _logger = logger; _blobDirectory = blobDirectory; _checkpointContainerName = checkpointContainerName; - if (inputType == Constants.InputType_Kafka || inputType == Constants.InputType_KafkaEventHub) + if (inputMode == Constants.InputMode_Batching) { - _messageBus = new KafkaMessageBus(brokerList, connectionString, hubNames, consumerGroup, inputType, logger); + _messageBus = new BlobMessageBus(batchInputs, logger); } else { - _messageBus = new EventhubMessageBus(hubNames[0], consumerGroup, connectionString, storageConnectionString, checkpointContainerName, inputType, logger); + if (inputType == Constants.InputType_Kafka || inputType == Constants.InputType_KafkaEventHub) + { + _messageBus = new KafkaMessageBus(brokerList, connectionString, hubNames, consumerGroup, inputType, logger); + } + else + { + _messageBus = new EventhubMessageBus(hubNames[0], consumerGroup, connectionString, storageConnectionString, checkpointContainerName, inputType, logger); + } + BlobHelper.DeleteAllBlobsInAContainer(storageConnectionString, checkpointContainerName, blobDirectory).Wait(); } - BlobHelper.DeleteAllBlobsInAContainer(storageConnectionString, checkpointContainerName, blobDirectory).Wait(); } /// diff --git a/Services/DataX.Flow/DataX.Flow.SchemaInference/SchemaInferenceManager.cs b/Services/DataX.Flow/DataX.Flow.SchemaInference/SchemaInferenceManager.cs index f492765f7..1e0185068 100644 --- a/Services/DataX.Flow/DataX.Flow.SchemaInference/SchemaInferenceManager.cs +++ b/Services/DataX.Flow/DataX.Flow.SchemaInference/SchemaInferenceManager.cs @@ -11,20 +11,24 @@ using Newtonsoft.Json.Linq; using System.Collections.Generic; using System.Threading.Tasks; - +using Microsoft.Extensions.Configuration; + namespace DataX.Flow.SchemaInference { public class SchemaInferenceManager { - private EngineEnvironment _engineEnvironment = new EngineEnvironment(); + private EngineEnvironment _engineEnvironment; private readonly ILogger _logger; + private readonly IConfiguration _configuration; private string OpsSamplePath => $@"https://{_engineEnvironment.EngineFlowConfig.OpsBlobBase}/samples"; - public SchemaInferenceManager(ILogger logger) + public SchemaInferenceManager(ILogger logger, IConfiguration configuration) { _logger = logger; - } + _configuration = configuration; + _engineEnvironment = new EngineEnvironment(_configuration); + } public async Task GetInputSchema(JObject jObject) { @@ -76,7 +80,7 @@ public async Task GetInputSchema(JObject jObject) var bootstrapServers = Helper.TryGetBootstrapServers(connectionString); // Sample events and generate schema - SchemaGenerator sg = new SchemaGenerator(bootstrapServers, connectionString, eventHubNames, _engineEnvironment.EngineFlowConfig.ConsumerGroup, _engineEnvironment.OpsBlobConnectionString, checkPointContainerName, _engineEnvironment.EngineFlowConfig.OpsBlobDirectory, diag.InputType, _logger); + SchemaGenerator sg = new SchemaGenerator(bootstrapServers, connectionString, eventHubNames, _engineEnvironment.EngineFlowConfig.ConsumerGroup, _engineEnvironment.OpsBlobConnectionString, checkPointContainerName, _engineEnvironment.EngineFlowConfig.OpsBlobDirectory, diag.InputType, diag.InputMode, diag.BatchInputs, _logger); SchemaResult schema = await sg.GetSchemaAsync(_engineEnvironment.OpsBlobConnectionString, OpsSamplePath, diag.UserName, diag.Name, diag.Seconds); return ApiResult.CreateSuccess(JObject.FromObject(schema)); @@ -132,7 +136,7 @@ public async Task RefreshSample(JObject jObject) var bootstrapServers = Helper.TryGetBootstrapServers(connectionString); // Sample events and refresh sample - SchemaGenerator sg = new SchemaGenerator(bootstrapServers, connectionString, eventHubNames, _engineEnvironment.EngineFlowConfig.ConsumerGroup, _engineEnvironment.OpsBlobConnectionString, checkPointContainerName, _engineEnvironment.EngineFlowConfig.OpsBlobDirectory, diag.InputType, _logger); + SchemaGenerator sg = new SchemaGenerator(bootstrapServers, connectionString, eventHubNames, _engineEnvironment.EngineFlowConfig.ConsumerGroup, _engineEnvironment.OpsBlobConnectionString, checkPointContainerName, _engineEnvironment.EngineFlowConfig.OpsBlobDirectory, diag.InputType, diag.InputMode, diag.BatchInputs, _logger); await sg.RefreshSample(_engineEnvironment.OpsBlobConnectionString, OpsSamplePath, diag.UserName, diag.Name, diag.Seconds); _logger.LogInformation("Refresh Sample worked!"); return ApiResult.CreateSuccess("success"); diff --git a/Services/DataX.Flow/DataX.Flow.SqlParser.Tests/DataX.Flow.SqlParser.Tests.csproj b/Services/DataX.Flow/DataX.Flow.SqlParser.Tests/DataX.Flow.SqlParser.Tests.csproj index 40dc6d977..ab107fd17 100644 --- a/Services/DataX.Flow/DataX.Flow.SqlParser.Tests/DataX.Flow.SqlParser.Tests.csproj +++ b/Services/DataX.Flow/DataX.Flow.SqlParser.Tests/DataX.Flow.SqlParser.Tests.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 false @@ -62,7 +62,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/DataX.Flow.SqlParser/DataX.Flow.SqlParser.csproj b/Services/DataX.Flow/DataX.Flow.SqlParser/DataX.Flow.SqlParser.csproj index 8e6a5cf60..15bd212a9 100644 --- a/Services/DataX.Flow/DataX.Flow.SqlParser/DataX.Flow.SqlParser.csproj +++ b/Services/DataX.Flow/DataX.Flow.SqlParser/DataX.Flow.SqlParser.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -25,7 +25,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/DataX.Flow.sln b/Services/DataX.Flow/DataX.Flow.sln index 63957d0e4..458db524f 100644 --- a/Services/DataX.Flow/DataX.Flow.sln +++ b/Services/DataX.Flow/DataX.Flow.sln @@ -1,448 +1,495 @@ - -Microsoft Visual Studio Solution File, Format Version 12.00 -# Visual Studio 15 -VisualStudioVersion = 15.0.28203.51 -MinimumVisualStudioVersion = 10.0.40219.1 -Project("{A07B5EB6-E848-4116-A8D0-A826331D98C6}") = "DataX.Flow", "DataX.Flow\DataX.Flow.sfproj", "{34E85436-8F73-44FC-9C03-CA205077D68D}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Flow.ManagementService", "Flow.ManagementService\Flow.ManagementService.csproj", "{3BA3102B-C8F3-468A-A06D-B6A02EE3B993}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.CodegenRules", "DataX.Flow.CodegenRules\DataX.Flow.CodegenRules.csproj", "{DBADD6FA-C068-412D-87F5-72CCE4361EF3}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.DeleteHelper", "DataX.Flow.DeleteHelper\DataX.Flow.DeleteHelper.csproj", "{D93F4C01-0F02-44DD-996C-DE99B085999B}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.SchemaInference", "DataX.Flow.SchemaInference\DataX.Flow.SchemaInference.csproj", "{338CB7A9-CA71-4D75-990D-3211B4A8CAE7}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.SqlParser", "DataX.Flow.SqlParser\DataX.Flow.SqlParser.csproj", "{A35DCE0E-167E-49FF-8024-8BC3B4BB7775}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.SchemaInference.Tests", "DataX.Flow.SchemaInference.Tests\DataX.Flow.SchemaInference.Tests.csproj", "{DC79570D-82AC-4240-BBFF-BF67298D9EAD}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.Blob", "..\DataX.Utilities\DataX.Utilities.Blob\DataX.Utilities.Blob.csproj", "{BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.KeyVault", "..\DataX.Utilities\DataX.Utilities.KeyVault\DataX.Utilities.KeyVault.csproj", "{D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.EventHub", "..\DataX.Utilities\DataX.Utilities.EventHub\DataX.Utilities.EventHub.csproj", "{A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.CosmosDB", "..\DataX.Utilities\DataX.Utilities.CosmosDB\DataX.Utilities.CosmosDB.csproj", "{19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Flow.InteractiveQueryService", "Flow.InteractiveQueryService\Flow.InteractiveQueryService.csproj", "{BC34FFEB-5D22-41DE-AFD1-B02D6632D966}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.InteractiveQuery", "DataX.Flow.InteractiveQuery\DataX.Flow.InteractiveQuery.csproj", "{7DE846AD-8ADE-49EA-80F4-E82240D73280}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Flow.SchemaInferenceService", "Flow.SchemaInferenceService\Flow.SchemaInferenceService.csproj", "{A248991B-1DCE-414F-8F69-4E15AE7B5BCD}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.Common", "DataX.Flow.Common\DataX.Flow.Common.csproj", "{4AF5863E-CFE1-4B6E-ADAA-B0031C839532}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Flow.LiveDataService", "Flow.LiveDataService\Flow.LiveDataService.csproj", "{434D84A0-F2AF-4F96-9BE6-7376F52D1A71}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.CodegenRules.Tests", "DataX.Flow.CodegenRules.Tests\DataX.Flow.CodegenRules.Tests.csproj", "{AE18E85F-CE02-473E-9DE3-B7A19069FF68}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.SqlParser.Tests", "DataX.Flow.SqlParser.Tests\DataX.Flow.SqlParser.Tests.csproj", "{C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.Telemetry", "..\DataX.Utilities\DataX.Utilities.Telemetry\DataX.Utilities.Telemetry.csproj", "{EF704744-E4A9-42D4-830A-17EEA66A98FC}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.Common.Tests", "DataX.Flow.Common.Tests\DataX.Flow.Common.Tests.csproj", "{16FD6E57-0B07-48A3-A505-2824220A10F2}" -EndProject -Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "DataX.Utilities", "DataX.Utilities", "{BAC65147-0FB3-457E-992E-0EEBA3DC9330}" -EndProject -Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "DataX.Contract", "DataX.Contract", "{7D216C6D-7895-41DF-92F3-B5F0E0472F3B}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Contract", "..\DataX.Contract\DataX.Contract.csproj", "{75E388EF-A3E7-4248-86D1-99782D97D9A1}" -EndProject -Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "DataX.ServiceHost", "DataX.ServiceHost", "{AFC65A75-34B4-4B0F-B447-757122588F26}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.ServiceHost.ServiceFabric", "..\DataX.ServiceHost\DataX.ServiceHost.ServiceFabric\DataX.ServiceHost.ServiceFabric.csproj", "{E7FDFA12-EBE1-4A09-B58B-9B95C575E53E}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.ServiceHost.AspNetCore", "..\DataX.ServiceHost\DataX.ServiceHost.AspNetCore\DataX.ServiceHost.AspNetCore.csproj", "{E682FD26-B182-4929-9C32-1FF931D71B77}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config", "..\DataX.Config\DataX.Config\DataX.Config.csproj", "{26D768EE-564E-482C-B3B5-A6A8F9937FD0}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.ConfigurationProviders", "..\DataX.Config\DataX.Config.ConfigurationProviders\DataX.Config.ConfigurationProviders.csproj", "{DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Storage", "..\DataX.Config\DataX.Config.Storage\DataX.Config.Storage.csproj", "{81DCD627-71D8-4F0D-B5F9-C9F8C6035442}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.KeyVault", "..\DataX.Config\DataX.Config.KeyVault\DataX.Config.KeyVault.csproj", "{EA3250C1-1895-4CE5-8361-AA7DA21E21EA}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.KeyVault", "..\DataX.Utilities\DataX.Utility.KeyVault\DataX.Utility.KeyVault.csproj", "{2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.Blob", "..\DataX.Utilities\DataX.Utility.Blob\DataX.Utility.Blob.csproj", "{66A5AD2F-FB6D-47D0-A405-35068A6B3897}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.CosmosDB", "..\DataX.Utilities\DataX.Utility.CosmosDB\DataX.Utility.CosmosDB.csproj", "{352A37F3-D9CA-4B8A-B455-9091FBE70D16}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Test", "..\DataX.Config\DataX.Config.Test\DataX.Config.Test.csproj", "{B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Test.Utility", "..\DataX.Config\DataX.Config.Test.Utility\DataX.Config.Test.Utility.csproj", "{1376E883-42CB-4573-9955-C52B6054FE16}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.LivyClient", "..\DataX.Config\DataX.Config.LivyClient\DataX.Config.LivyClient.csproj", "{563A6DAE-08C8-4366-B48F-D452376E0ECF}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Local", "..\DataX.Config\DataX.Config.Local\DataX.Config.Local.csproj", "{8E348951-91A9-41BF-942B-0747826573E1}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Local.Test", "..\DataX.Config\DataX.Config.Local.Test\DataX.Config.Local.Test.csproj", "{8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Input.EventHub", "..\DataX.Config\DataX.Config.Input.EventHub\DataX.Config.Input.EventHub.csproj", "{2086CBF7-42F5-4110-BFF3-54463D743E1F}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.EventHub", "..\DataX.Config\DataX.Utility.EventHub\DataX.Utility.EventHub.csproj", "{7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.Web", "..\DataX.Utilities\DataX.Utilities.Web\DataX.Utilities.Web.csproj", "{58740BCA-B537-4A61-9085-51F192A7106D}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Gateway.Contract", "..\DataX.Gateway\DataX.Gateway.Contract\DataX.Gateway.Contract.csproj", "{5903D72A-B458-4CA0-802E-241410E2ADE7}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.Nuget", "..\DataX.Utilities\DataX.Utility.Nuget\DataX.Utility.Nuget.csproj", "{7BE24BB5-3237-4403-921E-3CFC5A4A8E20}" -EndProject -Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Solution Items", "Solution Items", "{33E458B4-903F-4941-B82A-FCC741D56785}" -EndProject -Global - GlobalSection(SolutionConfigurationPlatforms) = preSolution - Debug|Any CPU = Debug|Any CPU - Debug|x64 = Debug|x64 - Release|Any CPU = Release|Any CPU - Release|x64 = Release|x64 - EndGlobalSection - GlobalSection(ProjectConfigurationPlatforms) = postSolution - {34E85436-8F73-44FC-9C03-CA205077D68D}.Debug|Any CPU.ActiveCfg = Debug|x64 - {34E85436-8F73-44FC-9C03-CA205077D68D}.Debug|x64.ActiveCfg = Debug|x64 - {34E85436-8F73-44FC-9C03-CA205077D68D}.Debug|x64.Build.0 = Debug|x64 - {34E85436-8F73-44FC-9C03-CA205077D68D}.Debug|x64.Deploy.0 = Debug|x64 - {34E85436-8F73-44FC-9C03-CA205077D68D}.Release|Any CPU.ActiveCfg = Release|x64 - {34E85436-8F73-44FC-9C03-CA205077D68D}.Release|x64.ActiveCfg = Release|x64 - {34E85436-8F73-44FC-9C03-CA205077D68D}.Release|x64.Build.0 = Release|x64 - {34E85436-8F73-44FC-9C03-CA205077D68D}.Release|x64.Deploy.0 = Release|x64 - {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Debug|Any CPU.Build.0 = Debug|Any CPU - {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Debug|x64.ActiveCfg = Debug|Any CPU - {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Debug|x64.Build.0 = Debug|Any CPU - {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Release|Any CPU.ActiveCfg = Release|Any CPU - {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Release|Any CPU.Build.0 = Release|Any CPU - {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Release|x64.ActiveCfg = Release|Any CPU - {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Release|x64.Build.0 = Release|Any CPU - {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Debug|Any CPU.Build.0 = Debug|Any CPU - {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Debug|x64.ActiveCfg = Debug|Any CPU - {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Debug|x64.Build.0 = Debug|Any CPU - {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Release|Any CPU.ActiveCfg = Release|Any CPU - {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Release|Any CPU.Build.0 = Release|Any CPU - {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Release|x64.ActiveCfg = Release|Any CPU - {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Release|x64.Build.0 = Release|Any CPU - {D93F4C01-0F02-44DD-996C-DE99B085999B}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {D93F4C01-0F02-44DD-996C-DE99B085999B}.Debug|Any CPU.Build.0 = Debug|Any CPU - {D93F4C01-0F02-44DD-996C-DE99B085999B}.Debug|x64.ActiveCfg = Debug|Any CPU - {D93F4C01-0F02-44DD-996C-DE99B085999B}.Debug|x64.Build.0 = Debug|Any CPU - {D93F4C01-0F02-44DD-996C-DE99B085999B}.Release|Any CPU.ActiveCfg = Release|Any CPU - {D93F4C01-0F02-44DD-996C-DE99B085999B}.Release|Any CPU.Build.0 = Release|Any CPU - {D93F4C01-0F02-44DD-996C-DE99B085999B}.Release|x64.ActiveCfg = Release|Any CPU - {D93F4C01-0F02-44DD-996C-DE99B085999B}.Release|x64.Build.0 = Release|Any CPU - {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Debug|Any CPU.Build.0 = Debug|Any CPU - {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Debug|x64.ActiveCfg = Debug|Any CPU - {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Debug|x64.Build.0 = Debug|Any CPU - {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Release|Any CPU.ActiveCfg = Release|Any CPU - {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Release|Any CPU.Build.0 = Release|Any CPU - {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Release|x64.ActiveCfg = Release|Any CPU - {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Release|x64.Build.0 = Release|Any CPU - {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Debug|Any CPU.Build.0 = Debug|Any CPU - {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Debug|x64.ActiveCfg = Debug|Any CPU - {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Debug|x64.Build.0 = Debug|Any CPU - {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Release|Any CPU.ActiveCfg = Release|Any CPU - {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Release|Any CPU.Build.0 = Release|Any CPU - {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Release|x64.ActiveCfg = Release|Any CPU - {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Release|x64.Build.0 = Release|Any CPU - {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Debug|Any CPU.Build.0 = Debug|Any CPU - {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Debug|x64.ActiveCfg = Debug|Any CPU - {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Debug|x64.Build.0 = Debug|Any CPU - {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Release|Any CPU.ActiveCfg = Release|Any CPU - {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Release|Any CPU.Build.0 = Release|Any CPU - {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Release|x64.ActiveCfg = Release|Any CPU - {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Release|x64.Build.0 = Release|Any CPU - {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Debug|Any CPU.Build.0 = Debug|Any CPU - {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Debug|x64.ActiveCfg = Debug|Any CPU - {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Debug|x64.Build.0 = Debug|Any CPU - {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Release|Any CPU.ActiveCfg = Release|Any CPU - {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Release|Any CPU.Build.0 = Release|Any CPU - {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Release|x64.ActiveCfg = Release|Any CPU - {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Release|x64.Build.0 = Release|Any CPU - {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Debug|Any CPU.Build.0 = Debug|Any CPU - {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Debug|x64.ActiveCfg = Debug|Any CPU - {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Debug|x64.Build.0 = Debug|Any CPU - {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Release|Any CPU.ActiveCfg = Release|Any CPU - {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Release|Any CPU.Build.0 = Release|Any CPU - {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Release|x64.ActiveCfg = Release|Any CPU - {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Release|x64.Build.0 = Release|Any CPU - {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Debug|Any CPU.Build.0 = Debug|Any CPU - {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Debug|x64.ActiveCfg = Debug|Any CPU - {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Debug|x64.Build.0 = Debug|Any CPU - {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Release|Any CPU.ActiveCfg = Release|Any CPU - {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Release|Any CPU.Build.0 = Release|Any CPU - {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Release|x64.ActiveCfg = Release|Any CPU - {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Release|x64.Build.0 = Release|Any CPU - {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Debug|Any CPU.Build.0 = Debug|Any CPU - {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Debug|x64.ActiveCfg = Debug|Any CPU - {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Debug|x64.Build.0 = Debug|Any CPU - {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Release|Any CPU.ActiveCfg = Release|Any CPU - {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Release|Any CPU.Build.0 = Release|Any CPU - {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Release|x64.ActiveCfg = Release|Any CPU - {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Release|x64.Build.0 = Release|Any CPU - {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Debug|Any CPU.Build.0 = Debug|Any CPU - {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Debug|x64.ActiveCfg = Debug|Any CPU - {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Debug|x64.Build.0 = Debug|Any CPU - {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Release|Any CPU.ActiveCfg = Release|Any CPU - {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Release|Any CPU.Build.0 = Release|Any CPU - {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Release|x64.ActiveCfg = Release|Any CPU - {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Release|x64.Build.0 = Release|Any CPU - {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Debug|Any CPU.Build.0 = Debug|Any CPU - {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Debug|x64.ActiveCfg = Debug|Any CPU - {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Debug|x64.Build.0 = Debug|Any CPU - {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Release|Any CPU.ActiveCfg = Release|Any CPU - {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Release|Any CPU.Build.0 = Release|Any CPU - {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Release|x64.ActiveCfg = Release|Any CPU - {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Release|x64.Build.0 = Release|Any CPU - {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Debug|Any CPU.Build.0 = Debug|Any CPU - {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Debug|x64.ActiveCfg = Debug|Any CPU - {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Debug|x64.Build.0 = Debug|Any CPU - {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Release|Any CPU.ActiveCfg = Release|Any CPU - {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Release|Any CPU.Build.0 = Release|Any CPU - {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Release|x64.ActiveCfg = Release|Any CPU - {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Release|x64.Build.0 = Release|Any CPU - {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Debug|Any CPU.Build.0 = Debug|Any CPU - {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Debug|x64.ActiveCfg = Debug|Any CPU - {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Debug|x64.Build.0 = Debug|Any CPU - {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Release|Any CPU.ActiveCfg = Release|Any CPU - {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Release|Any CPU.Build.0 = Release|Any CPU - {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Release|x64.ActiveCfg = Release|Any CPU - {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Release|x64.Build.0 = Release|Any CPU - {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Debug|Any CPU.Build.0 = Debug|Any CPU - {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Debug|x64.ActiveCfg = Debug|Any CPU - {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Debug|x64.Build.0 = Debug|Any CPU - {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Release|Any CPU.ActiveCfg = Release|Any CPU - {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Release|Any CPU.Build.0 = Release|Any CPU - {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Release|x64.ActiveCfg = Release|Any CPU - {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Release|x64.Build.0 = Release|Any CPU - {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Debug|Any CPU.Build.0 = Debug|Any CPU - {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Debug|x64.ActiveCfg = Debug|Any CPU - {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Debug|x64.Build.0 = Debug|Any CPU - {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Release|Any CPU.ActiveCfg = Release|Any CPU - {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Release|Any CPU.Build.0 = Release|Any CPU - {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Release|x64.ActiveCfg = Release|Any CPU - {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Release|x64.Build.0 = Release|Any CPU - {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Debug|Any CPU.Build.0 = Debug|Any CPU - {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Debug|x64.ActiveCfg = Debug|Any CPU - {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Debug|x64.Build.0 = Debug|Any CPU - {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Release|Any CPU.ActiveCfg = Release|Any CPU - {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Release|Any CPU.Build.0 = Release|Any CPU - {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Release|x64.ActiveCfg = Release|Any CPU - {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Release|x64.Build.0 = Release|Any CPU - {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Debug|Any CPU.Build.0 = Debug|Any CPU - {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Debug|x64.ActiveCfg = Debug|Any CPU - {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Debug|x64.Build.0 = Debug|Any CPU - {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Release|Any CPU.ActiveCfg = Release|Any CPU - {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Release|Any CPU.Build.0 = Release|Any CPU - {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Release|x64.ActiveCfg = Release|Any CPU - {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Release|x64.Build.0 = Release|Any CPU - {16FD6E57-0B07-48A3-A505-2824220A10F2}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {16FD6E57-0B07-48A3-A505-2824220A10F2}.Debug|Any CPU.Build.0 = Debug|Any CPU - {16FD6E57-0B07-48A3-A505-2824220A10F2}.Debug|x64.ActiveCfg = Debug|Any CPU - {16FD6E57-0B07-48A3-A505-2824220A10F2}.Debug|x64.Build.0 = Debug|Any CPU - {16FD6E57-0B07-48A3-A505-2824220A10F2}.Release|Any CPU.ActiveCfg = Release|Any CPU - {16FD6E57-0B07-48A3-A505-2824220A10F2}.Release|Any CPU.Build.0 = Release|Any CPU - {16FD6E57-0B07-48A3-A505-2824220A10F2}.Release|x64.ActiveCfg = Release|Any CPU - {16FD6E57-0B07-48A3-A505-2824220A10F2}.Release|x64.Build.0 = Release|Any CPU - {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Debug|Any CPU.Build.0 = Debug|Any CPU - {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Debug|x64.ActiveCfg = Debug|Any CPU - {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Debug|x64.Build.0 = Debug|Any CPU - {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Release|Any CPU.ActiveCfg = Release|Any CPU - {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Release|Any CPU.Build.0 = Release|Any CPU - {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Release|x64.ActiveCfg = Release|Any CPU - {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Release|x64.Build.0 = Release|Any CPU - {E7FDFA12-EBE1-4A09-B58B-9B95C575E53E}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {E7FDFA12-EBE1-4A09-B58B-9B95C575E53E}.Debug|Any CPU.Build.0 = Debug|Any CPU - {E7FDFA12-EBE1-4A09-B58B-9B95C575E53E}.Debug|x64.ActiveCfg = Debug|Any CPU - {E7FDFA12-EBE1-4A09-B58B-9B95C575E53E}.Debug|x64.Build.0 = Debug|Any CPU - {E7FDFA12-EBE1-4A09-B58B-9B95C575E53E}.Release|Any CPU.ActiveCfg = Release|Any CPU - {E7FDFA12-EBE1-4A09-B58B-9B95C575E53E}.Release|Any CPU.Build.0 = Release|Any CPU - {E7FDFA12-EBE1-4A09-B58B-9B95C575E53E}.Release|x64.ActiveCfg = Release|Any CPU - {E7FDFA12-EBE1-4A09-B58B-9B95C575E53E}.Release|x64.Build.0 = Release|Any CPU - {E682FD26-B182-4929-9C32-1FF931D71B77}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {E682FD26-B182-4929-9C32-1FF931D71B77}.Debug|Any CPU.Build.0 = Debug|Any CPU - {E682FD26-B182-4929-9C32-1FF931D71B77}.Debug|x64.ActiveCfg = Debug|Any CPU - {E682FD26-B182-4929-9C32-1FF931D71B77}.Debug|x64.Build.0 = Debug|Any CPU - {E682FD26-B182-4929-9C32-1FF931D71B77}.Release|Any CPU.ActiveCfg = Release|Any CPU - {E682FD26-B182-4929-9C32-1FF931D71B77}.Release|Any CPU.Build.0 = Release|Any CPU - {E682FD26-B182-4929-9C32-1FF931D71B77}.Release|x64.ActiveCfg = Release|Any CPU - {E682FD26-B182-4929-9C32-1FF931D71B77}.Release|x64.Build.0 = Release|Any CPU - {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Debug|Any CPU.Build.0 = Debug|Any CPU - {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Debug|x64.ActiveCfg = Debug|Any CPU - {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Debug|x64.Build.0 = Debug|Any CPU - {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Release|Any CPU.ActiveCfg = Release|Any CPU - {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Release|Any CPU.Build.0 = Release|Any CPU - {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Release|x64.ActiveCfg = Release|Any CPU - {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Release|x64.Build.0 = Release|Any CPU - {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Debug|Any CPU.Build.0 = Debug|Any CPU - {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Debug|x64.ActiveCfg = Debug|Any CPU - {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Debug|x64.Build.0 = Debug|Any CPU - {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Release|Any CPU.ActiveCfg = Release|Any CPU - {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Release|Any CPU.Build.0 = Release|Any CPU - {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Release|x64.ActiveCfg = Release|Any CPU - {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Release|x64.Build.0 = Release|Any CPU - {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Debug|Any CPU.Build.0 = Debug|Any CPU - {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Debug|x64.ActiveCfg = Debug|Any CPU - {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Debug|x64.Build.0 = Debug|Any CPU - {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Release|Any CPU.ActiveCfg = Release|Any CPU - {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Release|Any CPU.Build.0 = Release|Any CPU - {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Release|x64.ActiveCfg = Release|Any CPU - {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Release|x64.Build.0 = Release|Any CPU - {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Debug|Any CPU.Build.0 = Debug|Any CPU - {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Debug|x64.ActiveCfg = Debug|Any CPU - {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Debug|x64.Build.0 = Debug|Any CPU - {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Release|Any CPU.ActiveCfg = Release|Any CPU - {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Release|Any CPU.Build.0 = Release|Any CPU - {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Release|x64.ActiveCfg = Release|Any CPU - {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Release|x64.Build.0 = Release|Any CPU - {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Debug|Any CPU.Build.0 = Debug|Any CPU - {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Debug|x64.ActiveCfg = Debug|Any CPU - {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Debug|x64.Build.0 = Debug|Any CPU - {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Release|Any CPU.ActiveCfg = Release|Any CPU - {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Release|Any CPU.Build.0 = Release|Any CPU - {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Release|x64.ActiveCfg = Release|Any CPU - {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Release|x64.Build.0 = Release|Any CPU - {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Debug|Any CPU.Build.0 = Debug|Any CPU - {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Debug|x64.ActiveCfg = Debug|Any CPU - {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Debug|x64.Build.0 = Debug|Any CPU - {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Release|Any CPU.ActiveCfg = Release|Any CPU - {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Release|Any CPU.Build.0 = Release|Any CPU - {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Release|x64.ActiveCfg = Release|Any CPU - {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Release|x64.Build.0 = Release|Any CPU - {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Debug|Any CPU.Build.0 = Debug|Any CPU - {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Debug|x64.ActiveCfg = Debug|Any CPU - {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Debug|x64.Build.0 = Debug|Any CPU - {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Release|Any CPU.ActiveCfg = Release|Any CPU - {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Release|Any CPU.Build.0 = Release|Any CPU - {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Release|x64.ActiveCfg = Release|Any CPU - {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Release|x64.Build.0 = Release|Any CPU - {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Debug|Any CPU.Build.0 = Debug|Any CPU - {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Debug|x64.ActiveCfg = Debug|Any CPU - {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Debug|x64.Build.0 = Debug|Any CPU - {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Release|Any CPU.ActiveCfg = Release|Any CPU - {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Release|Any CPU.Build.0 = Release|Any CPU - {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Release|x64.ActiveCfg = Release|Any CPU - {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Release|x64.Build.0 = Release|Any CPU - {1376E883-42CB-4573-9955-C52B6054FE16}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {1376E883-42CB-4573-9955-C52B6054FE16}.Debug|Any CPU.Build.0 = Debug|Any CPU - {1376E883-42CB-4573-9955-C52B6054FE16}.Debug|x64.ActiveCfg = Debug|Any CPU - {1376E883-42CB-4573-9955-C52B6054FE16}.Debug|x64.Build.0 = Debug|Any CPU - {1376E883-42CB-4573-9955-C52B6054FE16}.Release|Any CPU.ActiveCfg = Release|Any CPU - {1376E883-42CB-4573-9955-C52B6054FE16}.Release|Any CPU.Build.0 = Release|Any CPU - {1376E883-42CB-4573-9955-C52B6054FE16}.Release|x64.ActiveCfg = Release|Any CPU - {1376E883-42CB-4573-9955-C52B6054FE16}.Release|x64.Build.0 = Release|Any CPU - {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Debug|Any CPU.Build.0 = Debug|Any CPU - {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Debug|x64.ActiveCfg = Debug|Any CPU - {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Debug|x64.Build.0 = Debug|Any CPU - {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Release|Any CPU.ActiveCfg = Release|Any CPU - {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Release|Any CPU.Build.0 = Release|Any CPU - {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Release|x64.ActiveCfg = Release|Any CPU - {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Release|x64.Build.0 = Release|Any CPU - {8E348951-91A9-41BF-942B-0747826573E1}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {8E348951-91A9-41BF-942B-0747826573E1}.Debug|Any CPU.Build.0 = Debug|Any CPU - {8E348951-91A9-41BF-942B-0747826573E1}.Debug|x64.ActiveCfg = Debug|Any CPU - {8E348951-91A9-41BF-942B-0747826573E1}.Debug|x64.Build.0 = Debug|Any CPU - {8E348951-91A9-41BF-942B-0747826573E1}.Release|Any CPU.ActiveCfg = Release|Any CPU - {8E348951-91A9-41BF-942B-0747826573E1}.Release|Any CPU.Build.0 = Release|Any CPU - {8E348951-91A9-41BF-942B-0747826573E1}.Release|x64.ActiveCfg = Release|Any CPU - {8E348951-91A9-41BF-942B-0747826573E1}.Release|x64.Build.0 = Release|Any CPU - {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Debug|Any CPU.Build.0 = Debug|Any CPU - {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Debug|x64.ActiveCfg = Debug|Any CPU - {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Debug|x64.Build.0 = Debug|Any CPU - {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Release|Any CPU.ActiveCfg = Release|Any CPU - {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Release|Any CPU.Build.0 = Release|Any CPU - {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Release|x64.ActiveCfg = Release|Any CPU - {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Release|x64.Build.0 = Release|Any CPU - {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Debug|Any CPU.Build.0 = Debug|Any CPU - {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Debug|x64.ActiveCfg = Debug|Any CPU - {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Debug|x64.Build.0 = Debug|Any CPU - {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Release|Any CPU.ActiveCfg = Release|Any CPU - {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Release|Any CPU.Build.0 = Release|Any CPU - {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Release|x64.ActiveCfg = Release|Any CPU - {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Release|x64.Build.0 = Release|Any CPU - {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Debug|Any CPU.Build.0 = Debug|Any CPU - {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Debug|x64.ActiveCfg = Debug|Any CPU - {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Debug|x64.Build.0 = Debug|Any CPU - {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Release|Any CPU.ActiveCfg = Release|Any CPU - {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Release|Any CPU.Build.0 = Release|Any CPU - {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Release|x64.ActiveCfg = Release|Any CPU - {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Release|x64.Build.0 = Release|Any CPU - {58740BCA-B537-4A61-9085-51F192A7106D}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {58740BCA-B537-4A61-9085-51F192A7106D}.Debug|Any CPU.Build.0 = Debug|Any CPU - {58740BCA-B537-4A61-9085-51F192A7106D}.Debug|x64.ActiveCfg = Debug|Any CPU - {58740BCA-B537-4A61-9085-51F192A7106D}.Debug|x64.Build.0 = Debug|Any CPU - {58740BCA-B537-4A61-9085-51F192A7106D}.Release|Any CPU.ActiveCfg = Release|Any CPU - {58740BCA-B537-4A61-9085-51F192A7106D}.Release|Any CPU.Build.0 = Release|Any CPU - {58740BCA-B537-4A61-9085-51F192A7106D}.Release|x64.ActiveCfg = Release|Any CPU - {58740BCA-B537-4A61-9085-51F192A7106D}.Release|x64.Build.0 = Release|Any CPU - {5903D72A-B458-4CA0-802E-241410E2ADE7}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {5903D72A-B458-4CA0-802E-241410E2ADE7}.Debug|Any CPU.Build.0 = Debug|Any CPU - {5903D72A-B458-4CA0-802E-241410E2ADE7}.Debug|x64.ActiveCfg = Debug|Any CPU - {5903D72A-B458-4CA0-802E-241410E2ADE7}.Debug|x64.Build.0 = Debug|Any CPU - {5903D72A-B458-4CA0-802E-241410E2ADE7}.Release|Any CPU.ActiveCfg = Release|Any CPU - {5903D72A-B458-4CA0-802E-241410E2ADE7}.Release|Any CPU.Build.0 = Release|Any CPU - {5903D72A-B458-4CA0-802E-241410E2ADE7}.Release|x64.ActiveCfg = Release|Any CPU - {5903D72A-B458-4CA0-802E-241410E2ADE7}.Release|x64.Build.0 = Release|Any CPU - {66459C39-A871-47AF-A69A-0A553893C8F2}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {66459C39-A871-47AF-A69A-0A553893C8F2}.Debug|Any CPU.Build.0 = Debug|Any CPU - {66459C39-A871-47AF-A69A-0A553893C8F2}.Debug|x64.ActiveCfg = Debug|Any CPU - {66459C39-A871-47AF-A69A-0A553893C8F2}.Debug|x64.Build.0 = Debug|Any CPU - {66459C39-A871-47AF-A69A-0A553893C8F2}.Release|Any CPU.ActiveCfg = Release|Any CPU - {66459C39-A871-47AF-A69A-0A553893C8F2}.Release|Any CPU.Build.0 = Release|Any CPU - {66459C39-A871-47AF-A69A-0A553893C8F2}.Release|x64.ActiveCfg = Release|Any CPU - {66459C39-A871-47AF-A69A-0A553893C8F2}.Release|x64.Build.0 = Release|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|Any CPU.Build.0 = Debug|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|x64.ActiveCfg = Debug|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|x64.Build.0 = Debug|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|Any CPU.ActiveCfg = Release|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|Any CPU.Build.0 = Release|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|x64.ActiveCfg = Release|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|x64.Build.0 = Release|Any CPU - EndGlobalSection - GlobalSection(SolutionProperties) = preSolution - HideSolutionNode = FALSE - EndGlobalSection - GlobalSection(NestedProjects) = preSolution - {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259} = {BAC65147-0FB3-457E-992E-0EEBA3DC9330} - {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E} = {BAC65147-0FB3-457E-992E-0EEBA3DC9330} - {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3} = {BAC65147-0FB3-457E-992E-0EEBA3DC9330} - {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76} = {BAC65147-0FB3-457E-992E-0EEBA3DC9330} - {EF704744-E4A9-42D4-830A-17EEA66A98FC} = {BAC65147-0FB3-457E-992E-0EEBA3DC9330} - {75E388EF-A3E7-4248-86D1-99782D97D9A1} = {7D216C6D-7895-41DF-92F3-B5F0E0472F3B} - {E7FDFA12-EBE1-4A09-B58B-9B95C575E53E} = {AFC65A75-34B4-4B0F-B447-757122588F26} - {E682FD26-B182-4929-9C32-1FF931D71B77} = {AFC65A75-34B4-4B0F-B447-757122588F26} - {58740BCA-B537-4A61-9085-51F192A7106D} = {BAC65147-0FB3-457E-992E-0EEBA3DC9330} - EndGlobalSection - GlobalSection(ExtensibilityGlobals) = postSolution - SolutionGuid = {05B0442E-EBAA-4578-98F2-16B0DD20C3DB} - EndGlobalSection -EndGlobal + +Microsoft Visual Studio Solution File, Format Version 12.00 +# Visual Studio 15 +VisualStudioVersion = 15.0.28203.51 +MinimumVisualStudioVersion = 10.0.40219.1 +Project("{A07B5EB6-E848-4116-A8D0-A826331D98C6}") = "DataX.Flow", "DataX.Flow\DataX.Flow.sfproj", "{34E85436-8F73-44FC-9C03-CA205077D68D}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Flow.ManagementService", "Flow.ManagementService\Flow.ManagementService.csproj", "{3BA3102B-C8F3-468A-A06D-B6A02EE3B993}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.CodegenRules", "DataX.Flow.CodegenRules\DataX.Flow.CodegenRules.csproj", "{DBADD6FA-C068-412D-87F5-72CCE4361EF3}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.DeleteHelper", "DataX.Flow.DeleteHelper\DataX.Flow.DeleteHelper.csproj", "{D93F4C01-0F02-44DD-996C-DE99B085999B}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.SchemaInference", "DataX.Flow.SchemaInference\DataX.Flow.SchemaInference.csproj", "{338CB7A9-CA71-4D75-990D-3211B4A8CAE7}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.SqlParser", "DataX.Flow.SqlParser\DataX.Flow.SqlParser.csproj", "{A35DCE0E-167E-49FF-8024-8BC3B4BB7775}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.SchemaInference.Tests", "DataX.Flow.SchemaInference.Tests\DataX.Flow.SchemaInference.Tests.csproj", "{DC79570D-82AC-4240-BBFF-BF67298D9EAD}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.Blob", "..\DataX.Utilities\DataX.Utilities.Blob\DataX.Utilities.Blob.csproj", "{BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.KeyVault", "..\DataX.Utilities\DataX.Utilities.KeyVault\DataX.Utilities.KeyVault.csproj", "{D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.EventHub", "..\DataX.Utilities\DataX.Utilities.EventHub\DataX.Utilities.EventHub.csproj", "{A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.CosmosDB", "..\DataX.Utilities\DataX.Utilities.CosmosDB\DataX.Utilities.CosmosDB.csproj", "{19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Flow.InteractiveQueryService", "Flow.InteractiveQueryService\Flow.InteractiveQueryService.csproj", "{BC34FFEB-5D22-41DE-AFD1-B02D6632D966}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.InteractiveQuery", "DataX.Flow.InteractiveQuery\DataX.Flow.InteractiveQuery.csproj", "{7DE846AD-8ADE-49EA-80F4-E82240D73280}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Flow.SchemaInferenceService", "Flow.SchemaInferenceService\Flow.SchemaInferenceService.csproj", "{A248991B-1DCE-414F-8F69-4E15AE7B5BCD}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.Common", "DataX.Flow.Common\DataX.Flow.Common.csproj", "{4AF5863E-CFE1-4B6E-ADAA-B0031C839532}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Flow.LiveDataService", "Flow.LiveDataService\Flow.LiveDataService.csproj", "{434D84A0-F2AF-4F96-9BE6-7376F52D1A71}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.CodegenRules.Tests", "DataX.Flow.CodegenRules.Tests\DataX.Flow.CodegenRules.Tests.csproj", "{AE18E85F-CE02-473E-9DE3-B7A19069FF68}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.SqlParser.Tests", "DataX.Flow.SqlParser.Tests\DataX.Flow.SqlParser.Tests.csproj", "{C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.Telemetry", "..\DataX.Utilities\DataX.Utilities.Telemetry\DataX.Utilities.Telemetry.csproj", "{EF704744-E4A9-42D4-830A-17EEA66A98FC}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.Common.Tests", "DataX.Flow.Common.Tests\DataX.Flow.Common.Tests.csproj", "{16FD6E57-0B07-48A3-A505-2824220A10F2}" +EndProject +Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "DataX.Utilities", "DataX.Utilities", "{BAC65147-0FB3-457E-992E-0EEBA3DC9330}" +EndProject +Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "DataX.Contract", "DataX.Contract", "{7D216C6D-7895-41DF-92F3-B5F0E0472F3B}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Contract", "..\DataX.Contract\DataX.Contract.csproj", "{75E388EF-A3E7-4248-86D1-99782D97D9A1}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config", "..\DataX.Config\DataX.Config\DataX.Config.csproj", "{26D768EE-564E-482C-B3B5-A6A8F9937FD0}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.ConfigurationProviders", "..\DataX.Config\DataX.Config.ConfigurationProviders\DataX.Config.ConfigurationProviders.csproj", "{DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Storage", "..\DataX.Config\DataX.Config.Storage\DataX.Config.Storage.csproj", "{81DCD627-71D8-4F0D-B5F9-C9F8C6035442}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.KeyVault", "..\DataX.Config\DataX.Config.KeyVault\DataX.Config.KeyVault.csproj", "{EA3250C1-1895-4CE5-8361-AA7DA21E21EA}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.KeyVault", "..\DataX.Utilities\DataX.Utility.KeyVault\DataX.Utility.KeyVault.csproj", "{2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.Blob", "..\DataX.Utilities\DataX.Utility.Blob\DataX.Utility.Blob.csproj", "{66A5AD2F-FB6D-47D0-A405-35068A6B3897}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.CosmosDB", "..\DataX.Utilities\DataX.Utility.CosmosDB\DataX.Utility.CosmosDB.csproj", "{352A37F3-D9CA-4B8A-B455-9091FBE70D16}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Test", "..\DataX.Config\DataX.Config.Test\DataX.Config.Test.csproj", "{B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Test.Utility", "..\DataX.Config\DataX.Config.Test.Utility\DataX.Config.Test.Utility.csproj", "{1376E883-42CB-4573-9955-C52B6054FE16}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.LivyClient", "..\DataX.Config\DataX.Config.LivyClient\DataX.Config.LivyClient.csproj", "{563A6DAE-08C8-4366-B48F-D452376E0ECF}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Local", "..\DataX.Config\DataX.Config.Local\DataX.Config.Local.csproj", "{8E348951-91A9-41BF-942B-0747826573E1}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Local.Test", "..\DataX.Config\DataX.Config.Local.Test\DataX.Config.Local.Test.csproj", "{8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Input.EventHub", "..\DataX.Config\DataX.Config.Input.EventHub\DataX.Config.Input.EventHub.csproj", "{2086CBF7-42F5-4110-BFF3-54463D743E1F}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.EventHub", "..\DataX.Config\DataX.Utility.EventHub\DataX.Utility.EventHub.csproj", "{7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.Web", "..\DataX.Utilities\DataX.Utilities.Web\DataX.Utilities.Web.csproj", "{58740BCA-B537-4A61-9085-51F192A7106D}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Gateway.Contract", "..\DataX.Gateway\DataX.Gateway.Contract\DataX.Gateway.Contract.csproj", "{5903D72A-B458-4CA0-802E-241410E2ADE7}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.Nuget", "..\DataX.Utilities\DataX.Utility.Nuget\DataX.Utility.Nuget.csproj", "{7BE24BB5-3237-4403-921E-3CFC5A4A8E20}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.ServiceHost", "..\DataX.ServiceHost\DataX.ServiceHost\DataX.ServiceHost.csproj", "{9FE363A8-7187-4F29-910C-AD2A845B48E0}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.Scheduler", "DataX.Flow.Scheduler\DataX.Flow.Scheduler.csproj", "{6841102C-9535-441D-8367-39E6E8CDCE04}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.ServiceCommunication", "DataX.Utility.ServiceCommunication\DataX.Utility.ServiceCommunication.csproj", "{D9F40BA0-EFB9-4546-8F83-FC390BE7EDA5}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.ServiceHost.AspNetCore", "..\DataX.ServiceHost\DataX.ServiceHost.AspNetCore\DataX.ServiceHost.AspNetCore.csproj", "{52E1724E-62DD-4CC2-A5E8-AD3947EC5C9F}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.ServiceHost.ServiceFabric", "..\DataX.ServiceHost\DataX.ServiceHost.ServiceFabric\DataX.ServiceHost.ServiceFabric.csproj", "{37675FE0-BFD5-434C-B1CF-473EA564EF32}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.DatabricksClient", "..\DataX.Config\DataX.Config.DatabricksClient\DataX.Config.DatabricksClient.csproj", "{DFAC7BC7-3B32-4EF8-8C75-C3E910CECEB4}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Flow.InteractiveQuery.Tests", "DataX.Flow.InteractiveQuery.Tests\DataX.Flow.InteractiveQuery.Tests.csproj", "{55E82F07-671B-4BBC-A8ED-F588EF181E82}" +EndProject +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "DataX.Utilities.Composition", "DataX.Utilities.Composition\DataX.Utilities.Composition.csproj", "{ABAA5FF5-828B-4959-BAE5-71DED33F4993}" +EndProject +Global + GlobalSection(SolutionConfigurationPlatforms) = preSolution + Debug|Any CPU = Debug|Any CPU + Debug|x64 = Debug|x64 + Release|Any CPU = Release|Any CPU + Release|x64 = Release|x64 + EndGlobalSection + GlobalSection(ProjectConfigurationPlatforms) = postSolution + {34E85436-8F73-44FC-9C03-CA205077D68D}.Debug|Any CPU.ActiveCfg = Debug|x64 + {34E85436-8F73-44FC-9C03-CA205077D68D}.Debug|x64.ActiveCfg = Debug|x64 + {34E85436-8F73-44FC-9C03-CA205077D68D}.Debug|x64.Build.0 = Debug|x64 + {34E85436-8F73-44FC-9C03-CA205077D68D}.Debug|x64.Deploy.0 = Debug|x64 + {34E85436-8F73-44FC-9C03-CA205077D68D}.Release|Any CPU.ActiveCfg = Release|x64 + {34E85436-8F73-44FC-9C03-CA205077D68D}.Release|x64.ActiveCfg = Release|x64 + {34E85436-8F73-44FC-9C03-CA205077D68D}.Release|x64.Build.0 = Release|x64 + {34E85436-8F73-44FC-9C03-CA205077D68D}.Release|x64.Deploy.0 = Release|x64 + {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Debug|Any CPU.Build.0 = Debug|Any CPU + {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Debug|x64.ActiveCfg = Debug|Any CPU + {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Debug|x64.Build.0 = Debug|Any CPU + {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Release|Any CPU.ActiveCfg = Release|Any CPU + {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Release|Any CPU.Build.0 = Release|Any CPU + {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Release|x64.ActiveCfg = Release|Any CPU + {3BA3102B-C8F3-468A-A06D-B6A02EE3B993}.Release|x64.Build.0 = Release|Any CPU + {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Debug|Any CPU.Build.0 = Debug|Any CPU + {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Debug|x64.ActiveCfg = Debug|Any CPU + {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Debug|x64.Build.0 = Debug|Any CPU + {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Release|Any CPU.ActiveCfg = Release|Any CPU + {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Release|Any CPU.Build.0 = Release|Any CPU + {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Release|x64.ActiveCfg = Release|Any CPU + {DBADD6FA-C068-412D-87F5-72CCE4361EF3}.Release|x64.Build.0 = Release|Any CPU + {D93F4C01-0F02-44DD-996C-DE99B085999B}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {D93F4C01-0F02-44DD-996C-DE99B085999B}.Debug|Any CPU.Build.0 = Debug|Any CPU + {D93F4C01-0F02-44DD-996C-DE99B085999B}.Debug|x64.ActiveCfg = Debug|Any CPU + {D93F4C01-0F02-44DD-996C-DE99B085999B}.Debug|x64.Build.0 = Debug|Any CPU + {D93F4C01-0F02-44DD-996C-DE99B085999B}.Release|Any CPU.ActiveCfg = Release|Any CPU + {D93F4C01-0F02-44DD-996C-DE99B085999B}.Release|Any CPU.Build.0 = Release|Any CPU + {D93F4C01-0F02-44DD-996C-DE99B085999B}.Release|x64.ActiveCfg = Release|Any CPU + {D93F4C01-0F02-44DD-996C-DE99B085999B}.Release|x64.Build.0 = Release|Any CPU + {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Debug|Any CPU.Build.0 = Debug|Any CPU + {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Debug|x64.ActiveCfg = Debug|Any CPU + {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Debug|x64.Build.0 = Debug|Any CPU + {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Release|Any CPU.ActiveCfg = Release|Any CPU + {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Release|Any CPU.Build.0 = Release|Any CPU + {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Release|x64.ActiveCfg = Release|Any CPU + {338CB7A9-CA71-4D75-990D-3211B4A8CAE7}.Release|x64.Build.0 = Release|Any CPU + {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Debug|Any CPU.Build.0 = Debug|Any CPU + {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Debug|x64.ActiveCfg = Debug|Any CPU + {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Debug|x64.Build.0 = Debug|Any CPU + {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Release|Any CPU.ActiveCfg = Release|Any CPU + {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Release|Any CPU.Build.0 = Release|Any CPU + {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Release|x64.ActiveCfg = Release|Any CPU + {A35DCE0E-167E-49FF-8024-8BC3B4BB7775}.Release|x64.Build.0 = Release|Any CPU + {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Debug|Any CPU.Build.0 = Debug|Any CPU + {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Debug|x64.ActiveCfg = Debug|Any CPU + {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Debug|x64.Build.0 = Debug|Any CPU + {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Release|Any CPU.ActiveCfg = Release|Any CPU + {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Release|Any CPU.Build.0 = Release|Any CPU + {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Release|x64.ActiveCfg = Release|Any CPU + {DC79570D-82AC-4240-BBFF-BF67298D9EAD}.Release|x64.Build.0 = Release|Any CPU + {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Debug|Any CPU.Build.0 = Debug|Any CPU + {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Debug|x64.ActiveCfg = Debug|Any CPU + {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Debug|x64.Build.0 = Debug|Any CPU + {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Release|Any CPU.ActiveCfg = Release|Any CPU + {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Release|Any CPU.Build.0 = Release|Any CPU + {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Release|x64.ActiveCfg = Release|Any CPU + {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259}.Release|x64.Build.0 = Release|Any CPU + {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Debug|Any CPU.Build.0 = Debug|Any CPU + {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Debug|x64.ActiveCfg = Debug|Any CPU + {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Debug|x64.Build.0 = Debug|Any CPU + {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Release|Any CPU.ActiveCfg = Release|Any CPU + {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Release|Any CPU.Build.0 = Release|Any CPU + {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Release|x64.ActiveCfg = Release|Any CPU + {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E}.Release|x64.Build.0 = Release|Any CPU + {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Debug|Any CPU.Build.0 = Debug|Any CPU + {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Debug|x64.ActiveCfg = Debug|Any CPU + {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Debug|x64.Build.0 = Debug|Any CPU + {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Release|Any CPU.ActiveCfg = Release|Any CPU + {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Release|Any CPU.Build.0 = Release|Any CPU + {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Release|x64.ActiveCfg = Release|Any CPU + {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3}.Release|x64.Build.0 = Release|Any CPU + {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Debug|Any CPU.Build.0 = Debug|Any CPU + {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Debug|x64.ActiveCfg = Debug|Any CPU + {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Debug|x64.Build.0 = Debug|Any CPU + {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Release|Any CPU.ActiveCfg = Release|Any CPU + {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Release|Any CPU.Build.0 = Release|Any CPU + {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Release|x64.ActiveCfg = Release|Any CPU + {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76}.Release|x64.Build.0 = Release|Any CPU + {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Debug|Any CPU.Build.0 = Debug|Any CPU + {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Debug|x64.ActiveCfg = Debug|Any CPU + {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Debug|x64.Build.0 = Debug|Any CPU + {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Release|Any CPU.ActiveCfg = Release|Any CPU + {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Release|Any CPU.Build.0 = Release|Any CPU + {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Release|x64.ActiveCfg = Release|Any CPU + {BC34FFEB-5D22-41DE-AFD1-B02D6632D966}.Release|x64.Build.0 = Release|Any CPU + {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Debug|Any CPU.Build.0 = Debug|Any CPU + {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Debug|x64.ActiveCfg = Debug|Any CPU + {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Debug|x64.Build.0 = Debug|Any CPU + {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Release|Any CPU.ActiveCfg = Release|Any CPU + {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Release|Any CPU.Build.0 = Release|Any CPU + {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Release|x64.ActiveCfg = Release|Any CPU + {7DE846AD-8ADE-49EA-80F4-E82240D73280}.Release|x64.Build.0 = Release|Any CPU + {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Debug|Any CPU.Build.0 = Debug|Any CPU + {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Debug|x64.ActiveCfg = Debug|Any CPU + {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Debug|x64.Build.0 = Debug|Any CPU + {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Release|Any CPU.ActiveCfg = Release|Any CPU + {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Release|Any CPU.Build.0 = Release|Any CPU + {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Release|x64.ActiveCfg = Release|Any CPU + {A248991B-1DCE-414F-8F69-4E15AE7B5BCD}.Release|x64.Build.0 = Release|Any CPU + {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Debug|Any CPU.Build.0 = Debug|Any CPU + {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Debug|x64.ActiveCfg = Debug|Any CPU + {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Debug|x64.Build.0 = Debug|Any CPU + {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Release|Any CPU.ActiveCfg = Release|Any CPU + {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Release|Any CPU.Build.0 = Release|Any CPU + {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Release|x64.ActiveCfg = Release|Any CPU + {4AF5863E-CFE1-4B6E-ADAA-B0031C839532}.Release|x64.Build.0 = Release|Any CPU + {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Debug|Any CPU.Build.0 = Debug|Any CPU + {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Debug|x64.ActiveCfg = Debug|Any CPU + {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Debug|x64.Build.0 = Debug|Any CPU + {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Release|Any CPU.ActiveCfg = Release|Any CPU + {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Release|Any CPU.Build.0 = Release|Any CPU + {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Release|x64.ActiveCfg = Release|Any CPU + {434D84A0-F2AF-4F96-9BE6-7376F52D1A71}.Release|x64.Build.0 = Release|Any CPU + {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Debug|Any CPU.Build.0 = Debug|Any CPU + {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Debug|x64.ActiveCfg = Debug|Any CPU + {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Debug|x64.Build.0 = Debug|Any CPU + {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Release|Any CPU.ActiveCfg = Release|Any CPU + {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Release|Any CPU.Build.0 = Release|Any CPU + {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Release|x64.ActiveCfg = Release|Any CPU + {AE18E85F-CE02-473E-9DE3-B7A19069FF68}.Release|x64.Build.0 = Release|Any CPU + {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Debug|Any CPU.Build.0 = Debug|Any CPU + {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Debug|x64.ActiveCfg = Debug|Any CPU + {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Debug|x64.Build.0 = Debug|Any CPU + {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Release|Any CPU.ActiveCfg = Release|Any CPU + {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Release|Any CPU.Build.0 = Release|Any CPU + {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Release|x64.ActiveCfg = Release|Any CPU + {C74DC460-CA6D-474F-8ED5-EE3CC6B0E8C7}.Release|x64.Build.0 = Release|Any CPU + {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Debug|Any CPU.Build.0 = Debug|Any CPU + {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Debug|x64.ActiveCfg = Debug|Any CPU + {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Debug|x64.Build.0 = Debug|Any CPU + {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Release|Any CPU.ActiveCfg = Release|Any CPU + {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Release|Any CPU.Build.0 = Release|Any CPU + {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Release|x64.ActiveCfg = Release|Any CPU + {EF704744-E4A9-42D4-830A-17EEA66A98FC}.Release|x64.Build.0 = Release|Any CPU + {16FD6E57-0B07-48A3-A505-2824220A10F2}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {16FD6E57-0B07-48A3-A505-2824220A10F2}.Debug|Any CPU.Build.0 = Debug|Any CPU + {16FD6E57-0B07-48A3-A505-2824220A10F2}.Debug|x64.ActiveCfg = Debug|Any CPU + {16FD6E57-0B07-48A3-A505-2824220A10F2}.Debug|x64.Build.0 = Debug|Any CPU + {16FD6E57-0B07-48A3-A505-2824220A10F2}.Release|Any CPU.ActiveCfg = Release|Any CPU + {16FD6E57-0B07-48A3-A505-2824220A10F2}.Release|Any CPU.Build.0 = Release|Any CPU + {16FD6E57-0B07-48A3-A505-2824220A10F2}.Release|x64.ActiveCfg = Release|Any CPU + {16FD6E57-0B07-48A3-A505-2824220A10F2}.Release|x64.Build.0 = Release|Any CPU + {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Debug|Any CPU.Build.0 = Debug|Any CPU + {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Debug|x64.ActiveCfg = Debug|Any CPU + {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Debug|x64.Build.0 = Debug|Any CPU + {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Release|Any CPU.ActiveCfg = Release|Any CPU + {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Release|Any CPU.Build.0 = Release|Any CPU + {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Release|x64.ActiveCfg = Release|Any CPU + {75E388EF-A3E7-4248-86D1-99782D97D9A1}.Release|x64.Build.0 = Release|Any CPU + {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Debug|Any CPU.Build.0 = Debug|Any CPU + {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Debug|x64.ActiveCfg = Debug|Any CPU + {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Debug|x64.Build.0 = Debug|Any CPU + {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Release|Any CPU.ActiveCfg = Release|Any CPU + {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Release|Any CPU.Build.0 = Release|Any CPU + {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Release|x64.ActiveCfg = Release|Any CPU + {26D768EE-564E-482C-B3B5-A6A8F9937FD0}.Release|x64.Build.0 = Release|Any CPU + {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Debug|Any CPU.Build.0 = Debug|Any CPU + {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Debug|x64.ActiveCfg = Debug|Any CPU + {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Debug|x64.Build.0 = Debug|Any CPU + {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Release|Any CPU.ActiveCfg = Release|Any CPU + {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Release|Any CPU.Build.0 = Release|Any CPU + {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Release|x64.ActiveCfg = Release|Any CPU + {DEDFCDE7-4885-4B00-AB2B-A6196503A9E8}.Release|x64.Build.0 = Release|Any CPU + {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Debug|Any CPU.Build.0 = Debug|Any CPU + {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Debug|x64.ActiveCfg = Debug|Any CPU + {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Debug|x64.Build.0 = Debug|Any CPU + {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Release|Any CPU.ActiveCfg = Release|Any CPU + {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Release|Any CPU.Build.0 = Release|Any CPU + {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Release|x64.ActiveCfg = Release|Any CPU + {81DCD627-71D8-4F0D-B5F9-C9F8C6035442}.Release|x64.Build.0 = Release|Any CPU + {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Debug|Any CPU.Build.0 = Debug|Any CPU + {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Debug|x64.ActiveCfg = Debug|Any CPU + {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Debug|x64.Build.0 = Debug|Any CPU + {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Release|Any CPU.ActiveCfg = Release|Any CPU + {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Release|Any CPU.Build.0 = Release|Any CPU + {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Release|x64.ActiveCfg = Release|Any CPU + {EA3250C1-1895-4CE5-8361-AA7DA21E21EA}.Release|x64.Build.0 = Release|Any CPU + {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Debug|Any CPU.Build.0 = Debug|Any CPU + {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Debug|x64.ActiveCfg = Debug|Any CPU + {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Debug|x64.Build.0 = Debug|Any CPU + {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Release|Any CPU.ActiveCfg = Release|Any CPU + {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Release|Any CPU.Build.0 = Release|Any CPU + {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Release|x64.ActiveCfg = Release|Any CPU + {2EA0CFB9-13BE-48DE-883C-7F5A123B04F7}.Release|x64.Build.0 = Release|Any CPU + {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Debug|Any CPU.Build.0 = Debug|Any CPU + {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Debug|x64.ActiveCfg = Debug|Any CPU + {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Debug|x64.Build.0 = Debug|Any CPU + {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Release|Any CPU.ActiveCfg = Release|Any CPU + {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Release|Any CPU.Build.0 = Release|Any CPU + {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Release|x64.ActiveCfg = Release|Any CPU + {66A5AD2F-FB6D-47D0-A405-35068A6B3897}.Release|x64.Build.0 = Release|Any CPU + {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Debug|Any CPU.Build.0 = Debug|Any CPU + {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Debug|x64.ActiveCfg = Debug|Any CPU + {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Debug|x64.Build.0 = Debug|Any CPU + {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Release|Any CPU.ActiveCfg = Release|Any CPU + {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Release|Any CPU.Build.0 = Release|Any CPU + {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Release|x64.ActiveCfg = Release|Any CPU + {352A37F3-D9CA-4B8A-B455-9091FBE70D16}.Release|x64.Build.0 = Release|Any CPU + {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Debug|Any CPU.Build.0 = Debug|Any CPU + {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Debug|x64.ActiveCfg = Debug|Any CPU + {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Debug|x64.Build.0 = Debug|Any CPU + {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Release|Any CPU.ActiveCfg = Release|Any CPU + {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Release|Any CPU.Build.0 = Release|Any CPU + {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Release|x64.ActiveCfg = Release|Any CPU + {B49BAEAE-BFAD-4B16-8C25-952C0332EEFB}.Release|x64.Build.0 = Release|Any CPU + {1376E883-42CB-4573-9955-C52B6054FE16}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {1376E883-42CB-4573-9955-C52B6054FE16}.Debug|Any CPU.Build.0 = Debug|Any CPU + {1376E883-42CB-4573-9955-C52B6054FE16}.Debug|x64.ActiveCfg = Debug|Any CPU + {1376E883-42CB-4573-9955-C52B6054FE16}.Debug|x64.Build.0 = Debug|Any CPU + {1376E883-42CB-4573-9955-C52B6054FE16}.Release|Any CPU.ActiveCfg = Release|Any CPU + {1376E883-42CB-4573-9955-C52B6054FE16}.Release|Any CPU.Build.0 = Release|Any CPU + {1376E883-42CB-4573-9955-C52B6054FE16}.Release|x64.ActiveCfg = Release|Any CPU + {1376E883-42CB-4573-9955-C52B6054FE16}.Release|x64.Build.0 = Release|Any CPU + {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Debug|Any CPU.Build.0 = Debug|Any CPU + {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Debug|x64.ActiveCfg = Debug|Any CPU + {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Debug|x64.Build.0 = Debug|Any CPU + {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Release|Any CPU.ActiveCfg = Release|Any CPU + {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Release|Any CPU.Build.0 = Release|Any CPU + {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Release|x64.ActiveCfg = Release|Any CPU + {563A6DAE-08C8-4366-B48F-D452376E0ECF}.Release|x64.Build.0 = Release|Any CPU + {8E348951-91A9-41BF-942B-0747826573E1}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {8E348951-91A9-41BF-942B-0747826573E1}.Debug|Any CPU.Build.0 = Debug|Any CPU + {8E348951-91A9-41BF-942B-0747826573E1}.Debug|x64.ActiveCfg = Debug|Any CPU + {8E348951-91A9-41BF-942B-0747826573E1}.Debug|x64.Build.0 = Debug|Any CPU + {8E348951-91A9-41BF-942B-0747826573E1}.Release|Any CPU.ActiveCfg = Release|Any CPU + {8E348951-91A9-41BF-942B-0747826573E1}.Release|Any CPU.Build.0 = Release|Any CPU + {8E348951-91A9-41BF-942B-0747826573E1}.Release|x64.ActiveCfg = Release|Any CPU + {8E348951-91A9-41BF-942B-0747826573E1}.Release|x64.Build.0 = Release|Any CPU + {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Debug|Any CPU.Build.0 = Debug|Any CPU + {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Debug|x64.ActiveCfg = Debug|Any CPU + {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Debug|x64.Build.0 = Debug|Any CPU + {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Release|Any CPU.ActiveCfg = Release|Any CPU + {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Release|Any CPU.Build.0 = Release|Any CPU + {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Release|x64.ActiveCfg = Release|Any CPU + {8B08C0EA-974D-47B1-9E32-EDAAB1C660B4}.Release|x64.Build.0 = Release|Any CPU + {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Debug|Any CPU.Build.0 = Debug|Any CPU + {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Debug|x64.ActiveCfg = Debug|Any CPU + {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Debug|x64.Build.0 = Debug|Any CPU + {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Release|Any CPU.ActiveCfg = Release|Any CPU + {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Release|Any CPU.Build.0 = Release|Any CPU + {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Release|x64.ActiveCfg = Release|Any CPU + {2086CBF7-42F5-4110-BFF3-54463D743E1F}.Release|x64.Build.0 = Release|Any CPU + {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Debug|Any CPU.Build.0 = Debug|Any CPU + {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Debug|x64.ActiveCfg = Debug|Any CPU + {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Debug|x64.Build.0 = Debug|Any CPU + {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Release|Any CPU.ActiveCfg = Release|Any CPU + {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Release|Any CPU.Build.0 = Release|Any CPU + {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Release|x64.ActiveCfg = Release|Any CPU + {7C471B0A-FDEC-44C5-B312-E95A6A43BFE4}.Release|x64.Build.0 = Release|Any CPU + {58740BCA-B537-4A61-9085-51F192A7106D}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {58740BCA-B537-4A61-9085-51F192A7106D}.Debug|Any CPU.Build.0 = Debug|Any CPU + {58740BCA-B537-4A61-9085-51F192A7106D}.Debug|x64.ActiveCfg = Debug|Any CPU + {58740BCA-B537-4A61-9085-51F192A7106D}.Debug|x64.Build.0 = Debug|Any CPU + {58740BCA-B537-4A61-9085-51F192A7106D}.Release|Any CPU.ActiveCfg = Release|Any CPU + {58740BCA-B537-4A61-9085-51F192A7106D}.Release|Any CPU.Build.0 = Release|Any CPU + {58740BCA-B537-4A61-9085-51F192A7106D}.Release|x64.ActiveCfg = Release|Any CPU + {58740BCA-B537-4A61-9085-51F192A7106D}.Release|x64.Build.0 = Release|Any CPU + {5903D72A-B458-4CA0-802E-241410E2ADE7}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {5903D72A-B458-4CA0-802E-241410E2ADE7}.Debug|Any CPU.Build.0 = Debug|Any CPU + {5903D72A-B458-4CA0-802E-241410E2ADE7}.Debug|x64.ActiveCfg = Debug|Any CPU + {5903D72A-B458-4CA0-802E-241410E2ADE7}.Debug|x64.Build.0 = Debug|Any CPU + {5903D72A-B458-4CA0-802E-241410E2ADE7}.Release|Any CPU.ActiveCfg = Release|Any CPU + {5903D72A-B458-4CA0-802E-241410E2ADE7}.Release|Any CPU.Build.0 = Release|Any CPU + {5903D72A-B458-4CA0-802E-241410E2ADE7}.Release|x64.ActiveCfg = Release|Any CPU + {5903D72A-B458-4CA0-802E-241410E2ADE7}.Release|x64.Build.0 = Release|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|Any CPU.Build.0 = Debug|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|x64.ActiveCfg = Debug|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|x64.Build.0 = Debug|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|Any CPU.ActiveCfg = Release|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|Any CPU.Build.0 = Release|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|x64.ActiveCfg = Release|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|x64.Build.0 = Release|Any CPU + {9FE363A8-7187-4F29-910C-AD2A845B48E0}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {9FE363A8-7187-4F29-910C-AD2A845B48E0}.Debug|Any CPU.Build.0 = Debug|Any CPU + {9FE363A8-7187-4F29-910C-AD2A845B48E0}.Debug|x64.ActiveCfg = Debug|Any CPU + {9FE363A8-7187-4F29-910C-AD2A845B48E0}.Debug|x64.Build.0 = Debug|Any CPU + {9FE363A8-7187-4F29-910C-AD2A845B48E0}.Release|Any CPU.ActiveCfg = Release|Any CPU + {9FE363A8-7187-4F29-910C-AD2A845B48E0}.Release|Any CPU.Build.0 = Release|Any CPU + {9FE363A8-7187-4F29-910C-AD2A845B48E0}.Release|x64.ActiveCfg = Release|Any CPU + {9FE363A8-7187-4F29-910C-AD2A845B48E0}.Release|x64.Build.0 = Release|Any CPU + {6841102C-9535-441D-8367-39E6E8CDCE04}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {6841102C-9535-441D-8367-39E6E8CDCE04}.Debug|Any CPU.Build.0 = Debug|Any CPU + {6841102C-9535-441D-8367-39E6E8CDCE04}.Debug|x64.ActiveCfg = Debug|Any CPU + {6841102C-9535-441D-8367-39E6E8CDCE04}.Debug|x64.Build.0 = Debug|Any CPU + {6841102C-9535-441D-8367-39E6E8CDCE04}.Release|Any CPU.ActiveCfg = Release|Any CPU + {6841102C-9535-441D-8367-39E6E8CDCE04}.Release|Any CPU.Build.0 = Release|Any CPU + {6841102C-9535-441D-8367-39E6E8CDCE04}.Release|x64.ActiveCfg = Release|Any CPU + {6841102C-9535-441D-8367-39E6E8CDCE04}.Release|x64.Build.0 = Release|Any CPU + {D9F40BA0-EFB9-4546-8F83-FC390BE7EDA5}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {D9F40BA0-EFB9-4546-8F83-FC390BE7EDA5}.Debug|Any CPU.Build.0 = Debug|Any CPU + {D9F40BA0-EFB9-4546-8F83-FC390BE7EDA5}.Debug|x64.ActiveCfg = Debug|Any CPU + {D9F40BA0-EFB9-4546-8F83-FC390BE7EDA5}.Debug|x64.Build.0 = Debug|Any CPU + {D9F40BA0-EFB9-4546-8F83-FC390BE7EDA5}.Release|Any CPU.ActiveCfg = Release|Any CPU + {D9F40BA0-EFB9-4546-8F83-FC390BE7EDA5}.Release|Any CPU.Build.0 = Release|Any CPU + {D9F40BA0-EFB9-4546-8F83-FC390BE7EDA5}.Release|x64.ActiveCfg = Release|Any CPU + {D9F40BA0-EFB9-4546-8F83-FC390BE7EDA5}.Release|x64.Build.0 = Release|Any CPU + {52E1724E-62DD-4CC2-A5E8-AD3947EC5C9F}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {52E1724E-62DD-4CC2-A5E8-AD3947EC5C9F}.Debug|Any CPU.Build.0 = Debug|Any CPU + {52E1724E-62DD-4CC2-A5E8-AD3947EC5C9F}.Debug|x64.ActiveCfg = Debug|Any CPU + {52E1724E-62DD-4CC2-A5E8-AD3947EC5C9F}.Debug|x64.Build.0 = Debug|Any CPU + {52E1724E-62DD-4CC2-A5E8-AD3947EC5C9F}.Release|Any CPU.ActiveCfg = Release|Any CPU + {52E1724E-62DD-4CC2-A5E8-AD3947EC5C9F}.Release|Any CPU.Build.0 = Release|Any CPU + {52E1724E-62DD-4CC2-A5E8-AD3947EC5C9F}.Release|x64.ActiveCfg = Release|Any CPU + {52E1724E-62DD-4CC2-A5E8-AD3947EC5C9F}.Release|x64.Build.0 = Release|Any CPU + {37675FE0-BFD5-434C-B1CF-473EA564EF32}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {37675FE0-BFD5-434C-B1CF-473EA564EF32}.Debug|Any CPU.Build.0 = Debug|Any CPU + {37675FE0-BFD5-434C-B1CF-473EA564EF32}.Debug|x64.ActiveCfg = Debug|Any CPU + {37675FE0-BFD5-434C-B1CF-473EA564EF32}.Debug|x64.Build.0 = Debug|Any CPU + {37675FE0-BFD5-434C-B1CF-473EA564EF32}.Release|Any CPU.ActiveCfg = Release|Any CPU + {37675FE0-BFD5-434C-B1CF-473EA564EF32}.Release|Any CPU.Build.0 = Release|Any CPU + {37675FE0-BFD5-434C-B1CF-473EA564EF32}.Release|x64.ActiveCfg = Release|Any CPU + {37675FE0-BFD5-434C-B1CF-473EA564EF32}.Release|x64.Build.0 = Release|Any CPU + {DFAC7BC7-3B32-4EF8-8C75-C3E910CECEB4}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {DFAC7BC7-3B32-4EF8-8C75-C3E910CECEB4}.Debug|Any CPU.Build.0 = Debug|Any CPU + {DFAC7BC7-3B32-4EF8-8C75-C3E910CECEB4}.Debug|x64.ActiveCfg = Debug|Any CPU + {DFAC7BC7-3B32-4EF8-8C75-C3E910CECEB4}.Debug|x64.Build.0 = Debug|Any CPU + {DFAC7BC7-3B32-4EF8-8C75-C3E910CECEB4}.Release|Any CPU.ActiveCfg = Release|Any CPU + {DFAC7BC7-3B32-4EF8-8C75-C3E910CECEB4}.Release|Any CPU.Build.0 = Release|Any CPU + {DFAC7BC7-3B32-4EF8-8C75-C3E910CECEB4}.Release|x64.ActiveCfg = Release|Any CPU + {DFAC7BC7-3B32-4EF8-8C75-C3E910CECEB4}.Release|x64.Build.0 = Release|Any CPU + {55E82F07-671B-4BBC-A8ED-F588EF181E82}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {55E82F07-671B-4BBC-A8ED-F588EF181E82}.Debug|Any CPU.Build.0 = Debug|Any CPU + {55E82F07-671B-4BBC-A8ED-F588EF181E82}.Debug|x64.ActiveCfg = Debug|Any CPU + {55E82F07-671B-4BBC-A8ED-F588EF181E82}.Debug|x64.Build.0 = Debug|Any CPU + {55E82F07-671B-4BBC-A8ED-F588EF181E82}.Release|Any CPU.ActiveCfg = Release|Any CPU + {55E82F07-671B-4BBC-A8ED-F588EF181E82}.Release|Any CPU.Build.0 = Release|Any CPU + {55E82F07-671B-4BBC-A8ED-F588EF181E82}.Release|x64.ActiveCfg = Release|Any CPU + {55E82F07-671B-4BBC-A8ED-F588EF181E82}.Release|x64.Build.0 = Release|Any CPU + {ABAA5FF5-828B-4959-BAE5-71DED33F4993}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {ABAA5FF5-828B-4959-BAE5-71DED33F4993}.Debug|Any CPU.Build.0 = Debug|Any CPU + {ABAA5FF5-828B-4959-BAE5-71DED33F4993}.Debug|x64.ActiveCfg = Debug|Any CPU + {ABAA5FF5-828B-4959-BAE5-71DED33F4993}.Debug|x64.Build.0 = Debug|Any CPU + {ABAA5FF5-828B-4959-BAE5-71DED33F4993}.Release|Any CPU.ActiveCfg = Release|Any CPU + {ABAA5FF5-828B-4959-BAE5-71DED33F4993}.Release|Any CPU.Build.0 = Release|Any CPU + {ABAA5FF5-828B-4959-BAE5-71DED33F4993}.Release|x64.ActiveCfg = Release|Any CPU + {ABAA5FF5-828B-4959-BAE5-71DED33F4993}.Release|x64.Build.0 = Release|Any CPU + EndGlobalSection + GlobalSection(SolutionProperties) = preSolution + HideSolutionNode = FALSE + EndGlobalSection + GlobalSection(NestedProjects) = preSolution + {BB5AE0A9-9100-419B-8EDA-27D7F7E8B259} = {BAC65147-0FB3-457E-992E-0EEBA3DC9330} + {D2FD4317-8BE6-4F1E-B5A8-CF766FE2E74E} = {BAC65147-0FB3-457E-992E-0EEBA3DC9330} + {A44D7D9E-A851-49A9-8F8E-9541FBB6A8D3} = {BAC65147-0FB3-457E-992E-0EEBA3DC9330} + {19AC676B-6FCE-4EA3-99A8-B893B9CC0B76} = {BAC65147-0FB3-457E-992E-0EEBA3DC9330} + {EF704744-E4A9-42D4-830A-17EEA66A98FC} = {BAC65147-0FB3-457E-992E-0EEBA3DC9330} + {75E388EF-A3E7-4248-86D1-99782D97D9A1} = {7D216C6D-7895-41DF-92F3-B5F0E0472F3B} + {58740BCA-B537-4A61-9085-51F192A7106D} = {BAC65147-0FB3-457E-992E-0EEBA3DC9330} + {ABAA5FF5-828B-4959-BAE5-71DED33F4993} = {BAC65147-0FB3-457E-992E-0EEBA3DC9330} + EndGlobalSection + GlobalSection(ExtensibilityGlobals) = postSolution + SolutionGuid = {05B0442E-EBAA-4578-98F2-16B0DD20C3DB} + EndGlobalSection +EndGlobal diff --git a/Services/DataX.Flow/DataX.Flow/ApplicationPackageRoot/ApplicationManifest.xml b/Services/DataX.Flow/DataX.Flow/ApplicationPackageRoot/ApplicationManifest.xml index 19799b071..6a6678a7f 100644 --- a/Services/DataX.Flow/DataX.Flow/ApplicationPackageRoot/ApplicationManifest.xml +++ b/Services/DataX.Flow/DataX.Flow/ApplicationPackageRoot/ApplicationManifest.xml @@ -1,142 +1,157 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
- - - - - - -
-
-
-
- - - - -
- - - - - -
- - - - - - -
-
-
-
- - - - -
- - - - - -
- - - - - -
-
-
-
- - - - -
- - - - - -
- - - - - - - - -
-
-
-
- - - - -
- - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ + + + + + + + + +
+
+
+
+ + + + +
+ + + + + +
+ + + + + + + + + +
+
+
+
+ + + + +
+ + + + + +
+ + + + + + + + +
+
+
+
+ + + + +
+ + + + + +
+ + + + + + + + + + + +
+
+
+
+ + + + +
+ + + + + + + + + + + + + + + + + + + + + + +
\ No newline at end of file diff --git a/Services/DataX.Flow/DataX.Flow/ApplicationParameters/Cloud.xml b/Services/DataX.Flow/DataX.Flow/ApplicationParameters/Cloud.xml index 3e3398f8c..035b5bdfc 100644 --- a/Services/DataX.Flow/DataX.Flow/ApplicationParameters/Cloud.xml +++ b/Services/DataX.Flow/DataX.Flow/ApplicationParameters/Cloud.xml @@ -1,23 +1,26 @@ - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/Services/DataX.Flow/DataX.Flow/ApplicationParameters/Local.1Node.xml b/Services/DataX.Flow/DataX.Flow/ApplicationParameters/Local.1Node.xml index 136c4b8d7..a7fcda37f 100644 --- a/Services/DataX.Flow/DataX.Flow/ApplicationParameters/Local.1Node.xml +++ b/Services/DataX.Flow/DataX.Flow/ApplicationParameters/Local.1Node.xml @@ -1,27 +1,30 @@ - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/Services/DataX.Flow/DataX.Flow/ApplicationParameters/Local.5Node.xml b/Services/DataX.Flow/DataX.Flow/ApplicationParameters/Local.5Node.xml index 136c4b8d7..a7fcda37f 100644 --- a/Services/DataX.Flow/DataX.Flow/ApplicationParameters/Local.5Node.xml +++ b/Services/DataX.Flow/DataX.Flow/ApplicationParameters/Local.5Node.xml @@ -1,27 +1,30 @@ - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/Services/DataX.Flow/DataX.Flow/DataX.Flow.sfproj b/Services/DataX.Flow/DataX.Flow/DataX.Flow.sfproj index a26023c28..a2a02ca91 100644 --- a/Services/DataX.Flow/DataX.Flow/DataX.Flow.sfproj +++ b/Services/DataX.Flow/DataX.Flow/DataX.Flow.sfproj @@ -1,50 +1,50 @@ - - - - - 34e85436-8f73-44fc-9c03-ca205077d68d - 2.3 - 1.5 - 1.6.7 - v4.6.1 - - - - Debug - x64 - - - Release - x64 - - - - - - - - - - - - - - - - - - - - - - - - $(MSBuildExtensionsPath32)\Microsoft\VisualStudio\v$(VisualStudioVersion)\Service Fabric Tools\Microsoft.VisualStudio.Azure.Fabric.ApplicationProject.targets - - - - - - - + + + + + 34e85436-8f73-44fc-9c03-ca205077d68d + 2.3 + 1.5 + 1.6.7 + v4.6.1 + + + + Debug + x64 + + + Release + x64 + + + + + + + + + + + + + + + + + + + + + + + + $(MSBuildExtensionsPath32)\Microsoft\VisualStudio\v$(VisualStudioVersion)\Service Fabric Tools\Microsoft.VisualStudio.Azure.Fabric.ApplicationProject.targets + + + + + + + \ No newline at end of file diff --git a/Services/DataX.Flow/DataX.Utilities.Composition/DataX.Utilities.Composition.csproj b/Services/DataX.Flow/DataX.Utilities.Composition/DataX.Utilities.Composition.csproj new file mode 100644 index 000000000..21258c328 --- /dev/null +++ b/Services/DataX.Flow/DataX.Utilities.Composition/DataX.Utilities.Composition.csproj @@ -0,0 +1,27 @@ + + + + netcoreapp2.2 + true + FinalPublicKey.snk + true + + + + + all + runtime; build; native; contentfiles; analyzers + + + all + runtime; build; native; contentfiles; analyzers + + + + + + Microsoft400 + StrongName + + + diff --git a/Services/DataX.Flow/DataX.Utilities.Composition/FinalPublicKey.snk b/Services/DataX.Flow/DataX.Utilities.Composition/FinalPublicKey.snk new file mode 100644 index 000000000..110b59c7b Binary files /dev/null and b/Services/DataX.Flow/DataX.Utilities.Composition/FinalPublicKey.snk differ diff --git a/Services/DataX.Flow/DataX.Utilities.Composition/InstanceExportDescriptorProvider.cs b/Services/DataX.Flow/DataX.Utilities.Composition/InstanceExportDescriptorProvider.cs new file mode 100644 index 000000000..b9ce97ac9 --- /dev/null +++ b/Services/DataX.Flow/DataX.Utilities.Composition/InstanceExportDescriptorProvider.cs @@ -0,0 +1,45 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using System; +using System.Collections.Generic; +using System.Composition.Hosting.Core; + +namespace DataX.Utilities.Composition +{ + /// + /// Custom export descriptor provider class to add existing instances to MEF container + /// + public class InstanceExportDescriptorProvider : ExportDescriptorProvider + { + private readonly TValue _exportedInstance; + + public InstanceExportDescriptorProvider(TValue exportedInstance) + { + _exportedInstance = exportedInstance; + } + + public override IEnumerable GetExportDescriptors(CompositionContract contract, DependencyAccessor descriptorAccessor) + { + object[] obj = null; + Type type = _exportedInstance.GetType(); + if (!type.IsArray) + { + obj = new object[] { _exportedInstance }; + } + else + { + obj = (object[])Convert.ChangeType(_exportedInstance, typeof(object[])); + } + foreach (var instance in obj) + { + if (contract.ContractType.IsInstanceOfType(instance)) + { + yield return new ExportDescriptorPromise(contract, instance.ToString(), true, NoDependencies, _ => + ExportDescriptor.Create((c, o) => instance, new Dictionary())); + } + } + } + } +} diff --git a/Services/DataX.Flow/DataX.Utility.ServiceCommunication/DataX.Utility.ServiceCommunication.csproj b/Services/DataX.Flow/DataX.Utility.ServiceCommunication/DataX.Utility.ServiceCommunication.csproj new file mode 100644 index 000000000..a59a0e743 --- /dev/null +++ b/Services/DataX.Flow/DataX.Utility.ServiceCommunication/DataX.Utility.ServiceCommunication.csproj @@ -0,0 +1,33 @@ + + + + netcoreapp2.2 + true + FinalPublicKey.snk + true + + + + + all + runtime; build; native; contentfiles; analyzers + + + all + runtime; build; native; contentfiles; analyzers + + + + + + + + + + + Microsoft400 + StrongName + + + + diff --git a/Services/DataX.Flow/DataX.Utility.ServiceCommunication/FinalPublicKey.snk b/Services/DataX.Flow/DataX.Utility.ServiceCommunication/FinalPublicKey.snk new file mode 100644 index 000000000..110b59c7b Binary files /dev/null and b/Services/DataX.Flow/DataX.Utility.ServiceCommunication/FinalPublicKey.snk differ diff --git a/Services/DataX.Flow/DataX.Utility.ServiceCommunication/InterServiceCommunicator.cs b/Services/DataX.Flow/DataX.Utility.ServiceCommunication/InterServiceCommunicator.cs new file mode 100644 index 000000000..a0ff842a1 --- /dev/null +++ b/Services/DataX.Flow/DataX.Utility.ServiceCommunication/InterServiceCommunicator.cs @@ -0,0 +1,167 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Contract; +using Newtonsoft.Json.Linq; +using System; +using System.Collections.Generic; +using System.Net.Http; +using System.Text; +using System.Threading.Tasks; + +namespace DataX.Utility.ServiceCommunication +{ + /// + /// Concrete implementation of inter-service communication client. + /// + public class InterServiceCommunicator + { + private const string _ReverseProxyPort = "19081"; + private readonly HttpClient _httpClient; + + public InterServiceCommunicator(TimeSpan timeout) + { + var handler = new WinHttpHandler + { + ServerCertificateValidationCallback = (message, cert, chain, errors) => { return true; }, + // We need to set timeout for handler first + // Setting timeout for httpClient alone is not good enough + SendTimeout = timeout, + ReceiveDataTimeout = timeout, + ReceiveHeadersTimeout = timeout + }; + + _httpClient = new HttpClient(handler) + { + // Default http timeout is 100s, increase it to 4 min since few key mainline scenarios + // can take longer than default 100s + Timeout = timeout + }; + } + + /// + /// Make request using reverse proxy + /// + /// + public virtual async Task InvokeServiceAsync(HttpMethod httpMethod, string application, string service, string method, Dictionary headers = null, string content = null) + { + return await InvokeServiceAsAsyncHelper(httpMethod, application, service, method, headers, content); + } + + /// + /// Make request Helper + /// + /// + private async Task InvokeServiceAsAsyncHelper(HttpMethod httpMethod, string application, string service, string method, Dictionary headers, string content) + { + var serviceUri = new Uri($"https://localhost:{_ReverseProxyPort}/{application}/{service}/"); + var apiUri = new UriBuilder(new Uri(serviceUri, $"api/{method}")); + + HttpRequestMessage request = new HttpRequestMessage + { + RequestUri = apiUri.Uri, + Method = httpMethod + }; + + if (!string.IsNullOrEmpty(content)) + { + request.Content = new StringContent(content, Encoding.UTF8, "application/json"); + request.Headers.Add("Content-type", "application/json"); + } + + if (headers != null) + { + foreach(var header in headers) + { + request.Headers.Add(header.Key, header.Value); + + } + } + + HttpResponseMessage response = null; + try + { + response = await _httpClient.SendAsync(request); + } + catch (Exception) + { + throw; + } + + var result = new ApiResult(); + ProxyResponse(result, response); + + return result; + } + + /// + /// Proxy response message + /// + /// + private async void ProxyResponse(ApiResult result, HttpResponseMessage response) + { + var content = await response.Content.ReadAsStringAsync(); + + // Handle the case where content is empty and preserve the original error reason (else will result in json parse error) + if (string.IsNullOrEmpty(content)) + { + result.Error = !response.IsSuccessStatusCode; + result.Message = response.ReasonPhrase; + } + else + { + try + { + var responseObj = JObject.Parse(content); + var errorProp = (bool?)responseObj["error"]; + var resultProp = responseObj["result"]; + var isError = errorProp.HasValue && errorProp.Value; + + if (response.IsSuccessStatusCode && !isError) + { + result.Result = resultProp; + } + else + { + result.Error = true; + string message = "Error response from service"; + + try + { + if (isError) + { + message = (string)responseObj["message"]; + } + // handle the case that the exception came directly from WebAPI + else if (responseObj["ExceptionMessage"] != null) + { + message = (string)responseObj["ExceptionMessage"]; + } + else + { + message = (string)responseObj["Message"]; + } + } + catch + { + // ignore errors + } + + result.Message = message; + + if (isError && resultProp != null) + { + result.Result = resultProp; + } + } + } + catch (Exception e) + { + result.Error = true; + result.Message = $"Unable to parse content. Error={e.Message}"; + } + } + } + } +} diff --git a/Services/DataX.Flow/Flow.InteractiveQueryService/Controllers/InteractiveQueryController.cs b/Services/DataX.Flow/Flow.InteractiveQueryService/Controllers/InteractiveQueryController.cs index ca0e440f8..f26f15fa7 100644 --- a/Services/DataX.Flow/Flow.InteractiveQueryService/Controllers/InteractiveQueryController.cs +++ b/Services/DataX.Flow/Flow.InteractiveQueryService/Controllers/InteractiveQueryController.cs @@ -2,28 +2,34 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using DataX.Contract; +using DataX.Flow.Common.Models; +using DataX.Flow.InteractiveQuery; +using DataX.ServiceHost.AspNetCore.Authorization.Roles; +using DataX.Utilities.Web; using Microsoft.AspNetCore.Mvc; +using Microsoft.Extensions.Configuration; using Microsoft.Extensions.Logging; using Newtonsoft.Json.Linq; -using DataX.Contract; -//using DataX.Flow.Common; -using DataX.Flow.InteractiveQuery; using System; using System.Threading.Tasks; -using DataX.Utilities.Web; - -// For more information on enabling Web API for empty projects, visit https://go.microsoft.com/fwlink/?LinkID=397860 - + +// For more information on enabling Web API for empty projects, visit https://go.microsoft.com/fwlink/?LinkID=397860 + namespace Flow.InteractiveQueryService.Controllers { [Route("api")] + [DataXWriter] public class InteractiveQueryController : Controller { private readonly ILogger _logger; - public InteractiveQueryController(ILogger logger) + private readonly IConfiguration _configuration; + public InteractiveQueryController(ILogger logger, IConfiguration configuration) { _logger = logger; + _configuration = configuration; } + [HttpPost] [Route("kernel")] // diag public async Task CreateAndInitializeKernel([FromBody]JObject jObject) @@ -31,7 +37,7 @@ public async Task CreateAndInitializeKernel([FromBody]JObject jObject try { RolesCheck.EnsureWriter(Request); - InteractiveQueryManager iqm = new InteractiveQueryManager(_logger); + InteractiveQueryManager iqm = new InteractiveQueryManager(_logger, _configuration); ApiResult response = await iqm.CreateAndInitializeKernel(jObject); //Logging information / success @@ -54,7 +60,7 @@ public async Task RefreshKernel([FromBody]JObject jObject) { RolesCheck.EnsureWriter(Request); - InteractiveQueryManager iqm = new InteractiveQueryManager(_logger); + InteractiveQueryManager iqm = new InteractiveQueryManager(_logger, _configuration); return await iqm.RecycleKernel(jObject); } @@ -67,15 +73,15 @@ public async Task RefreshKernel([FromBody]JObject jObject) [HttpPost] [Route("kernel/deleteList")] - public async Task DeleteKernelList([FromBody]System.Collections.Generic.List kernels) + public async Task DeleteKernelList([FromBody]System.Collections.Generic.List kernels, string flowName) { try { RolesCheck.EnsureWriter(Request); - InteractiveQueryManager iqm = new InteractiveQueryManager(_logger); - return await iqm.DeleteKernelList(kernels).ConfigureAwait(false); + InteractiveQueryManager iqm = new InteractiveQueryManager(_logger, _configuration); + return await iqm.DeleteKernelList(kernels, flowName).ConfigureAwait(false); } catch (Exception e) { @@ -86,15 +92,15 @@ public async Task DeleteKernelList([FromBody]System.Collections.Gener [HttpPost] [Route("kernel/delete")] // diag - public async Task DeleteKernel([FromBody]string kernelId) + public async Task DeleteKernel([FromBody]JObject jObject) { try { RolesCheck.EnsureWriter(Request); - - InteractiveQueryManager iqm = new InteractiveQueryManager(_logger); - return await iqm.DeleteKernel(kernelId).ConfigureAwait(false); + var diag = jObject.ToObject(); + InteractiveQueryManager iqm = new InteractiveQueryManager(_logger, _configuration); + return await iqm.DeleteKernel(diag.KernelId, diag.Name).ConfigureAwait(false); } catch (Exception e) @@ -106,14 +112,14 @@ public async Task DeleteKernel([FromBody]string kernelId) [HttpPost] [Route("kernels/delete")] - public async Task DeleteKernels() + public async Task DeleteKernels([FromBody]string flowName) { try { RolesCheck.EnsureWriter(Request); - InteractiveQueryManager iqm = new InteractiveQueryManager(_logger); - return await iqm.DeleteKernels().ConfigureAwait(false); + InteractiveQueryManager iqm = new InteractiveQueryManager(_logger, _configuration); + return await iqm.DeleteKernels(flowName).ConfigureAwait(false); } catch (Exception e) @@ -125,14 +131,14 @@ public async Task DeleteKernels() [HttpPost] [Route("kernels/deleteall")] - public async Task DeleteAllKernels() + public async Task DeleteAllKernels([FromBody]string flowName) { try { RolesCheck.EnsureWriter(Request); - InteractiveQueryManager iqm = new InteractiveQueryManager(_logger); - return await iqm.DeleteAllKernels().ConfigureAwait(false); + InteractiveQueryManager iqm = new InteractiveQueryManager(_logger, _configuration); + return await iqm.DeleteAllKernels(flowName).ConfigureAwait(false); } catch (Exception e) { @@ -150,7 +156,7 @@ public async Task GetSampleInput([FromBody]JObject jObject) { RolesCheck.EnsureWriter(Request); - InteractiveQueryManager iqm = new InteractiveQueryManager(_logger); + InteractiveQueryManager iqm = new InteractiveQueryManager(_logger, _configuration); return await iqm.GetSampleInputFromQuery(jObject); } @@ -168,7 +174,7 @@ public async Task ExecuteCode([FromBody]JObject jObject) try { RolesCheck.EnsureWriter(Request); - InteractiveQueryManager iqm = new InteractiveQueryManager(_logger); + InteractiveQueryManager iqm = new InteractiveQueryManager(_logger, _configuration); return await iqm.ExecuteQuery(jObject); } diff --git a/Services/DataX.Flow/Flow.InteractiveQueryService/Flow.InteractiveQueryService.csproj b/Services/DataX.Flow/Flow.InteractiveQueryService/Flow.InteractiveQueryService.csproj index 2e4402585..928875d6d 100644 --- a/Services/DataX.Flow/Flow.InteractiveQueryService/Flow.InteractiveQueryService.csproj +++ b/Services/DataX.Flow/Flow.InteractiveQueryService/Flow.InteractiveQueryService.csproj @@ -1,7 +1,7 @@ - netcoreapp2.1 + netcoreapp2.2 True win7-x64 False @@ -25,14 +25,13 @@ - - - + + all runtime; build; native; contentfiles; analyzers - + all @@ -61,14 +60,14 @@ - Microsoft + Microsoft400 StrongName - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/Flow.InteractiveQueryService/FlowInteractiveQueryServiceStartup.cs b/Services/DataX.Flow/Flow.InteractiveQueryService/FlowInteractiveQueryServiceStartup.cs new file mode 100644 index 000000000..0640dc323 --- /dev/null +++ b/Services/DataX.Flow/Flow.InteractiveQueryService/FlowInteractiveQueryServiceStartup.cs @@ -0,0 +1,24 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +namespace Flow.InteractiveQueryService +{ + using Microsoft.AspNetCore.Hosting; + using DataX.ServiceHost.AspNetCore; + using DataX.ServiceHost.AspNetCore.Startup; + using DataX.Contract.Settings; + + /// + /// StartupFilter for Flow.InteractiveQueryService + /// + public class FlowInteractiveQueryServiceStartup : DataXServiceStartup + { + public FlowInteractiveQueryServiceStartup() { } + + public FlowInteractiveQueryServiceStartup(DataXSettings settings) + : base(settings) { } + } +} + + diff --git a/Services/DataX.Flow/Flow.InteractiveQueryService/InteractiveQueryService.cs b/Services/DataX.Flow/Flow.InteractiveQueryService/InteractiveQueryService.cs index 1e767bddd..8057b8b53 100644 --- a/Services/DataX.Flow/Flow.InteractiveQueryService/InteractiveQueryService.cs +++ b/Services/DataX.Flow/Flow.InteractiveQueryService/InteractiveQueryService.cs @@ -26,9 +26,13 @@ namespace Flow.InteractiveQueryService ///
internal sealed class InteractiveQueryService : StatelessService { - public InteractiveQueryService(StatelessServiceContext context) + private readonly IWebHostBuilder _hostBuilder; + + public InteractiveQueryService(StatelessServiceContext context, IWebHostBuilder hostBuilder) : base(context) - { } + { + _hostBuilder = hostBuilder; + } /// /// Optional override to create listeners (like tcp, http) for this service instance. @@ -42,13 +46,10 @@ protected override IEnumerable CreateServiceInstanceLis new KestrelCommunicationListener(serviceContext, "ServiceEndpoint", (url, listener) => { ServiceEventSource.Current.ServiceMessage(serviceContext, $"Starting Kestrel on {url}"); - return new WebHostBuilder() - .UseKestrel() + return _hostBuilder .ConfigureServices( services => services .AddSingleton(serviceContext)) - .UseContentRoot(Directory.GetCurrentDirectory()) - .UseStartup() .UseServiceFabricIntegration(listener, ServiceFabricIntegrationOptions.None) .UseUrls(url) .Build(); diff --git a/Services/DataX.Flow/Flow.InteractiveQueryService/PackageRoot/Config/Settings.xml b/Services/DataX.Flow/Flow.InteractiveQueryService/PackageRoot/Config/Settings.xml index 088bf9c6d..7bf110dad 100644 --- a/Services/DataX.Flow/Flow.InteractiveQueryService/PackageRoot/Config/Settings.xml +++ b/Services/DataX.Flow/Flow.InteractiveQueryService/PackageRoot/Config/Settings.xml @@ -11,6 +11,9 @@ Licensed under the MIT License + + + diff --git a/Services/DataX.Flow/Flow.InteractiveQueryService/Program.cs b/Services/DataX.Flow/Flow.InteractiveQueryService/Program.cs index f97156b43..df1d55cef 100644 --- a/Services/DataX.Flow/Flow.InteractiveQueryService/Program.cs +++ b/Services/DataX.Flow/Flow.InteractiveQueryService/Program.cs @@ -2,6 +2,9 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using DataX.ServiceHost; +using DataX.ServiceHost.AspNetCore.Extensions; +using Microsoft.AspNetCore.Hosting; using Microsoft.ServiceFabric.Services.Runtime; using System; using System.Diagnostics; @@ -19,18 +22,28 @@ private static void Main() { try { - // The ServiceManifest.XML file defines one or more service type names. - // Registering a service maps a service type name to a .NET type. - // When Service Fabric creates an instance of this service type, - // an instance of the class is created in this host process. + if (HostUtil.InServiceFabric) + { + // The ServiceManifest.XML file defines one or more service type names. + // Registering a service maps a service type name to a .NET type. + // When Service Fabric creates an instance of this service type, + // an instance of the class is created in this host process. - ServiceRuntime.RegisterServiceAsync("Flow.InteractiveQueryServiceType", - context => new InteractiveQueryService(context)).GetAwaiter().GetResult(); + ServiceRuntime.RegisterServiceAsync("Flow.InteractiveQueryServiceType", + context => new InteractiveQueryService(context, WebHostBuilder)).GetAwaiter().GetResult(); - ServiceEventSource.Current.ServiceTypeRegistered(Process.GetCurrentProcess().Id, typeof(InteractiveQueryService).Name); + ServiceEventSource.Current.ServiceTypeRegistered(Process.GetCurrentProcess().Id, typeof(InteractiveQueryService).Name); - // Prevents this host process from terminating so services keeps running. - Thread.Sleep(Timeout.Infinite); + // Prevents this host process from terminating so services keeps running. + Thread.Sleep(Timeout.Infinite); + } + else + { + var webHost = WebHostBuilder.Build(); + + webHost.Start(); + webHost.WaitForShutdown(); + } } catch (Exception e) { @@ -38,5 +51,13 @@ private static void Main() throw; } } + + /// + /// Create a new WebHostBuilder with DataX default configuration that allows this to be run standalone + /// + private static IWebHostBuilder WebHostBuilder + => new WebHostBuilder() + .UseDataXDefaultConfiguration() + .Configure(app => app.UseDataXApplicationDefaults()); } } diff --git a/Services/DataX.Flow/Flow.InteractiveQueryService/appsettings.Development.json b/Services/DataX.Flow/Flow.InteractiveQueryService/appsettings.Development.json index e203e9407..bcb491cc8 100644 --- a/Services/DataX.Flow/Flow.InteractiveQueryService/appsettings.Development.json +++ b/Services/DataX.Flow/Flow.InteractiveQueryService/appsettings.Development.json @@ -5,5 +5,31 @@ "System": "Information", "Microsoft": "Information" } + }, + "Kestrel": { + "EndPoints": { + "Http": { + "Url": "http://localhost:5000" + }, + "HttpsDefaultCert": { + "Url": "https://localhost:5001", + "Protocols": "Http1AndHttp2" + } + } + }, + "DataX": { + "ServiceEnvironment": { + "EnableOneBox": false, + "AppInsightsIntrumentationKey": "", + "CosmosDBConfigCollectionName": "", + "CosmosDBConfigConnectionString": "", + "CosmosDBConfigDatabaseName": "", + "ServiceKeyVaultName": "", + "TestClientId": "" + } + }, + "JwtBearerOptions": { + "Audience": "", + "Authority": null // JWT ISS (issuer) } -} +} \ No newline at end of file diff --git a/Services/DataX.Flow/Flow.InteractiveQueryService/appsettings.json b/Services/DataX.Flow/Flow.InteractiveQueryService/appsettings.json index d524c85fa..f9211d461 100644 --- a/Services/DataX.Flow/Flow.InteractiveQueryService/appsettings.json +++ b/Services/DataX.Flow/Flow.InteractiveQueryService/appsettings.json @@ -1,9 +1,23 @@ { "Logging": { - "IncludeScopes": false, "LogLevel": { "Default": "Information" } }, - "AllowedHosts": "*" - } \ No newline at end of file + "AllowedHosts": "*", + "DataX": { + "ServiceEnvironment": { + "EnableOneBox": false, + "AppInsightsIntrumentationKey": "", + "CosmosDBConfigCollectionName": "", + "CosmosDBConfigConnectionString": "", + "CosmosDBConfigDatabaseName": "", + "TestClientId": "", + "ServiceKeyVaultName": "" + } + }, + "JwtBearerOptions": { + "Audience": "", + "Authority": null // JWT ISS (issuer) + } +} diff --git a/Services/DataX.Flow/Flow.LiveDataService/Controllers/LiveDataController.cs b/Services/DataX.Flow/Flow.LiveDataService/Controllers/LiveDataController.cs index 13c69f6f0..cf0db1678 100644 --- a/Services/DataX.Flow/Flow.LiveDataService/Controllers/LiveDataController.cs +++ b/Services/DataX.Flow/Flow.LiveDataService/Controllers/LiveDataController.cs @@ -9,16 +9,21 @@ using System; using System.Threading.Tasks; using DataX.Utilities.Web; +using DataX.ServiceHost.AspNetCore.Authorization.Roles; +using Microsoft.Extensions.Configuration; namespace Flow.LiveDataService.Controllers { [Route("api")] + [DataXWriter] public class LiveDataController : Controller { private readonly ILogger _logger; - public LiveDataController(ILogger logger) + private readonly IConfiguration _configuration; + public LiveDataController(ILogger logger, IConfiguration configuration) { _logger = logger; + _configuration = configuration; } [HttpPost] @@ -28,7 +33,7 @@ public async Task RefreshInputDataAndKernel([FromBody]JObject jObject try { RolesCheck.EnsureWriter(Request); - LiveDataManager ldm = new LiveDataManager(_logger); + LiveDataManager ldm = new LiveDataManager(_logger, _configuration); return await ldm.RefreshInputDataAndKernel(jObject); } catch (Exception e) diff --git a/Services/DataX.Flow/Flow.LiveDataService/Flow.LiveDataService.csproj b/Services/DataX.Flow/Flow.LiveDataService/Flow.LiveDataService.csproj index a09a1a196..87a8494a1 100644 --- a/Services/DataX.Flow/Flow.LiveDataService/Flow.LiveDataService.csproj +++ b/Services/DataX.Flow/Flow.LiveDataService/Flow.LiveDataService.csproj @@ -1,7 +1,7 @@ - + - netcoreapp2.1 + netcoreapp2.2 True win7-x64 False @@ -25,15 +25,14 @@ - - - + + all runtime; build; native; contentfiles; analyzers - - + + all @@ -55,14 +54,14 @@ - Microsoft + Microsoft400 StrongName - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/Flow.LiveDataService/FlowLiveDataServiceStartup.cs b/Services/DataX.Flow/Flow.LiveDataService/FlowLiveDataServiceStartup.cs new file mode 100644 index 000000000..2b68b7271 --- /dev/null +++ b/Services/DataX.Flow/Flow.LiveDataService/FlowLiveDataServiceStartup.cs @@ -0,0 +1,23 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +namespace Flow.LiveDataService +{ + using Microsoft.AspNetCore.Hosting; + using DataX.ServiceHost.AspNetCore; + using DataX.ServiceHost.AspNetCore.Startup; + using DataX.Contract.Settings; + + /// + /// StartupFilter for Flow.LiveDataService + /// + public class FlowLiveDataServiceStartup : DataXServiceStartup + { + public FlowLiveDataServiceStartup() { } + public FlowLiveDataServiceStartup(DataXSettings settings) + : base(settings) { } + } +} + + diff --git a/Services/DataX.Flow/Flow.LiveDataService/LiveDataManager.cs b/Services/DataX.Flow/Flow.LiveDataService/LiveDataManager.cs index e328a51c0..25cf518eb 100644 --- a/Services/DataX.Flow/Flow.LiveDataService/LiveDataManager.cs +++ b/Services/DataX.Flow/Flow.LiveDataService/LiveDataManager.cs @@ -10,18 +10,22 @@ using DataX.Flow.InteractiveQuery; using DataX.Flow.SchemaInference; using System.Threading.Tasks; - +using Microsoft.Extensions.Configuration; + namespace Flow.LiveDataService { //TODO: Change the name to represent InteractiveQuery and generateSchema combined together public class LiveDataManager { private string _flowContainerName => _engineEnvironment.EngineFlowConfig.FlowContainerName; - private EngineEnvironment _engineEnvironment = new EngineEnvironment(); + private EngineEnvironment _engineEnvironment; private readonly ILogger _logger; - public LiveDataManager(ILogger logger) + private readonly IConfiguration _configuration; + public LiveDataManager(ILogger logger, IConfiguration configuration) { _logger = logger; + _configuration = configuration; + _engineEnvironment = new EngineEnvironment(_configuration); } public async Task RefreshInputDataAndKernel(JObject jObject) { @@ -35,7 +39,7 @@ public async Task RefreshInputDataAndKernel(JObject jObject) } //Refresh the sample data - SchemaInferenceManager sim = new SchemaInferenceManager(_logger); + SchemaInferenceManager sim = new SchemaInferenceManager(_logger, _configuration); response = await sim.RefreshSample(jObject); if (response.Error.HasValue && response.Error.Value) @@ -44,7 +48,7 @@ public async Task RefreshInputDataAndKernel(JObject jObject) return ApiResult.CreateError(response.Message); } - InteractiveQueryManager iqm = new InteractiveQueryManager(_logger); + InteractiveQueryManager iqm = new InteractiveQueryManager(_logger, _configuration); response = await iqm.RecycleKernelHelper(diag, true); if (response.Error.HasValue && response.Error.Value) { diff --git a/Services/DataX.Flow/Flow.LiveDataService/LiveDataService.cs b/Services/DataX.Flow/Flow.LiveDataService/LiveDataService.cs index 26a51afe4..ca5725ea6 100644 --- a/Services/DataX.Flow/Flow.LiveDataService/LiveDataService.cs +++ b/Services/DataX.Flow/Flow.LiveDataService/LiveDataService.cs @@ -26,9 +26,13 @@ namespace Flow.LiveDataService /// internal sealed class LiveDataService : StatelessService { - public LiveDataService(StatelessServiceContext context) + private readonly IWebHostBuilder _hostBuilder; + + public LiveDataService(StatelessServiceContext context, IWebHostBuilder hostBuilder) : base(context) - { } + { + _hostBuilder = hostBuilder; + } /// /// Optional override to create listeners (like tcp, http) for this service instance. @@ -43,13 +47,10 @@ protected override IEnumerable CreateServiceInstanceLis { ServiceEventSource.Current.ServiceMessage(serviceContext, $"Starting Kestrel on {url}"); - return new WebHostBuilder() - .UseKestrel() + return _hostBuilder .ConfigureServices( services => services .AddSingleton(serviceContext)) - .UseContentRoot(Directory.GetCurrentDirectory()) - .UseStartup() .UseServiceFabricIntegration(listener, ServiceFabricIntegrationOptions.None) .UseUrls(url) .Build(); diff --git a/Services/DataX.Flow/Flow.LiveDataService/PackageRoot/Config/Settings.xml b/Services/DataX.Flow/Flow.LiveDataService/PackageRoot/Config/Settings.xml index b4d8ee23f..12da835a2 100644 --- a/Services/DataX.Flow/Flow.LiveDataService/PackageRoot/Config/Settings.xml +++ b/Services/DataX.Flow/Flow.LiveDataService/PackageRoot/Config/Settings.xml @@ -12,5 +12,8 @@ Licensed under the MIT License + + + diff --git a/Services/DataX.Flow/Flow.LiveDataService/Program.cs b/Services/DataX.Flow/Flow.LiveDataService/Program.cs index e01d66a84..28b28b833 100644 --- a/Services/DataX.Flow/Flow.LiveDataService/Program.cs +++ b/Services/DataX.Flow/Flow.LiveDataService/Program.cs @@ -2,6 +2,9 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using DataX.ServiceHost; +using DataX.ServiceHost.AspNetCore.Extensions; +using Microsoft.AspNetCore.Hosting; using Microsoft.ServiceFabric.Services.Runtime; using System; using System.Diagnostics; @@ -19,18 +22,28 @@ private static void Main() { try { - // The ServiceManifest.XML file defines one or more service type names. - // Registering a service maps a service type name to a .NET type. - // When Service Fabric creates an instance of this service type, - // an instance of the class is created in this host process. + if(HostUtil.InServiceFabric) + { + // The ServiceManifest.XML file defines one or more service type names. + // Registering a service maps a service type name to a .NET type. + // When Service Fabric creates an instance of this service type, + // an instance of the class is created in this host process. - ServiceRuntime.RegisterServiceAsync("Flow.LiveDataServiceType", - context => new LiveDataService(context)).GetAwaiter().GetResult(); + ServiceRuntime.RegisterServiceAsync("Flow.LiveDataServiceType", + context => new LiveDataService(context, WebHostBuilder)).GetAwaiter().GetResult(); - ServiceEventSource.Current.ServiceTypeRegistered(Process.GetCurrentProcess().Id, typeof(LiveDataService).Name); + ServiceEventSource.Current.ServiceTypeRegistered(Process.GetCurrentProcess().Id, typeof(LiveDataService).Name); - // Prevents this host process from terminating so services keeps running. - Thread.Sleep(Timeout.Infinite); + // Prevents this host process from terminating so services keeps running. + Thread.Sleep(Timeout.Infinite); + } + else + { + var webHost = WebHostBuilder.Build(); + + webHost.Start(); + webHost.WaitForShutdown(); + } } catch (Exception e) { @@ -38,5 +51,13 @@ private static void Main() throw; } } + + /// + /// Create a new WebHostBuilder with DataX default configuration that allows this to be run standalone + /// + private static IWebHostBuilder WebHostBuilder + => new WebHostBuilder() + .UseDataXDefaultConfiguration() + .Configure(app => app.UseDataXApplicationDefaults()); } } diff --git a/Services/DataX.Flow/Flow.LiveDataService/appsettings.Development.json b/Services/DataX.Flow/Flow.LiveDataService/appsettings.Development.json index e203e9407..bcb491cc8 100644 --- a/Services/DataX.Flow/Flow.LiveDataService/appsettings.Development.json +++ b/Services/DataX.Flow/Flow.LiveDataService/appsettings.Development.json @@ -5,5 +5,31 @@ "System": "Information", "Microsoft": "Information" } + }, + "Kestrel": { + "EndPoints": { + "Http": { + "Url": "http://localhost:5000" + }, + "HttpsDefaultCert": { + "Url": "https://localhost:5001", + "Protocols": "Http1AndHttp2" + } + } + }, + "DataX": { + "ServiceEnvironment": { + "EnableOneBox": false, + "AppInsightsIntrumentationKey": "", + "CosmosDBConfigCollectionName": "", + "CosmosDBConfigConnectionString": "", + "CosmosDBConfigDatabaseName": "", + "ServiceKeyVaultName": "", + "TestClientId": "" + } + }, + "JwtBearerOptions": { + "Audience": "", + "Authority": null // JWT ISS (issuer) } -} +} \ No newline at end of file diff --git a/Services/DataX.Flow/Flow.LiveDataService/appsettings.json b/Services/DataX.Flow/Flow.LiveDataService/appsettings.json index 6a845cfd4..f9211d461 100644 --- a/Services/DataX.Flow/Flow.LiveDataService/appsettings.json +++ b/Services/DataX.Flow/Flow.LiveDataService/appsettings.json @@ -4,5 +4,20 @@ "Default": "Information" } }, - "AllowedHosts": "*" + "AllowedHosts": "*", + "DataX": { + "ServiceEnvironment": { + "EnableOneBox": false, + "AppInsightsIntrumentationKey": "", + "CosmosDBConfigCollectionName": "", + "CosmosDBConfigConnectionString": "", + "CosmosDBConfigDatabaseName": "", + "TestClientId": "", + "ServiceKeyVaultName": "" + } + }, + "JwtBearerOptions": { + "Audience": "", + "Authority": null // JWT ISS (issuer) + } } diff --git a/Services/DataX.Flow/Flow.ManagementService/Controllers/FlowManagementController.cs b/Services/DataX.Flow/Flow.ManagementService/Controllers/FlowManagementController.cs index 3cfa30726..406e04887 100644 --- a/Services/DataX.Flow/Flow.ManagementService/Controllers/FlowManagementController.cs +++ b/Services/DataX.Flow/Flow.ManagementService/Controllers/FlowManagementController.cs @@ -17,21 +17,27 @@ using System.Linq; using DataX.Utilities.Web; using System.Collections.Generic; - +using Microsoft.AspNetCore.Authorization; +using DataX.ServiceHost.AspNetCore.Authorization.Roles; +using Microsoft.Extensions.Configuration; + namespace Flow.Management.Controllers { [Route("api")] + [DataXReader] public partial class FlowManagementController : Controller { private readonly ILogger _logger; - private FlowOperation _flowOperation; + private readonly IConfiguration _configuration; + private readonly FlowOperation _flowOperation; private JobOperation _jobOperation; private RuntimeConfigGeneration _runtimeConfigGenerator; private bool _isLocal = false; - public FlowManagementController(ILogger logger, FlowOperation flowOp, RuntimeConfigGeneration runtimeConfigGenerator, JobOperation jobOp) + public FlowManagementController(ILogger logger, IConfiguration configuration, FlowOperation flowOp, RuntimeConfigGeneration runtimeConfigGenerator, JobOperation jobOp) { _logger = logger; + _configuration = configuration; _flowOperation = flowOp; _jobOperation = jobOp; _runtimeConfigGenerator = runtimeConfigGenerator; @@ -44,6 +50,7 @@ public FlowManagementController(ILogger logger, FlowOp [HttpPost] [Route("flow/save")] // save flow config + [DataXWriter] public async Task SaveFlow([FromBody]JObject config) { try @@ -73,8 +80,28 @@ public async Task SaveFlow([FromBody]JObject config) } } + + [HttpPost] + [Route("flow/schedulebatch")] // schedule batch jobs + public async Task ScheduleBatch() + { + try + { + RolesCheck.EnsureWriter(Request, _isLocal); + var result = await _flowOperation.ScheduleBatch(this._runtimeConfigGenerator).ConfigureAwait(false); + + return ApiResult.CreateSuccess(JToken.FromObject(result)); + } + catch (Exception e) + { + _logger.LogError(e, e.Message); + return ApiResult.CreateError(e.Message); + } + } + [HttpPost] [Route("flow/generateconfigs")] // generate flow configs + [DataXWriter] public async Task GenerateConfigs([FromBody] string flowName) { try @@ -141,7 +168,7 @@ public async Task GetAllFlowsMin() RolesCheck.EnsureReader(Request, _isLocal); var flowConfigs = await _flowOperation.GetAllFlows(); - var result = flowConfigs.Select(x => new { name = x.Name, displayName = x.DisplayName, owner = x.GetGuiConfig().Owner }); + var result = flowConfigs.Select(x => new { name = x.Name, displayName = x.DisplayName, owner = x.GetGuiConfig()?.Owner }); return ApiResult.CreateSuccess(JToken.FromObject(result)); } catch (Exception e) @@ -154,6 +181,7 @@ public async Task GetAllFlowsMin() [HttpPost] [Route("flow/startjobs")] + [DataXWriter] public async Task StartJobsForFlow([FromBody] string flowName) { try @@ -174,6 +202,7 @@ public async Task StartJobsForFlow([FromBody] string flowName) [HttpPost] [Route("flow/restartjobs")] + [DataXWriter] public async Task RestartJobsForFlow([FromBody] string flowName) { try @@ -194,6 +223,7 @@ public async Task RestartJobsForFlow([FromBody] string flowName) [HttpPost] [Route("flow/stopjobs")] + [DataXWriter] public async Task StopJobsForFlow([FromBody] string flowName) { try @@ -215,6 +245,7 @@ public async Task StopJobsForFlow([FromBody] string flowName) [HttpPost] [Route("userqueries/schema")] // generator (sqlparser) + [DataXWriter] public async Task GetSchema([FromBody]JObject config) { try @@ -242,6 +273,7 @@ public async Task GetSchema([FromBody]JObject config) [HttpPost] [Route("userqueries/codegen")] // generator + [DataXWriter] public async Task GetCodeGen([FromBody]JObject config) { try @@ -328,6 +360,7 @@ public async Task GetJobsByNames([FromBody] string[] jobNames) [HttpPost] [Route("job/start")] // start the job + [DataXWriter] public async Task StartJob([FromBody] string jobName) { try @@ -349,6 +382,7 @@ public async Task StartJob([FromBody] string jobName) [HttpPost] [Route("job/stop")] // stop the job + [DataXWriter] public async Task StopJob([FromBody] string jobName) { try @@ -369,6 +403,7 @@ public async Task StopJob([FromBody] string jobName) [HttpPost] [Route("job/restart")] + [DataXWriter] public async Task RestartJob([FromBody] string jobName) { try @@ -390,6 +425,7 @@ public async Task RestartJob([FromBody] string jobName) [HttpGet] [Route("job/syncall")] + [DataXWriter] public async Task SyncAllJobs() { try diff --git a/Services/DataX.Flow/Flow.ManagementService/Controllers/FlowManagementControllerDelete.cs b/Services/DataX.Flow/Flow.ManagementService/Controllers/FlowManagementControllerDelete.cs index a29035956..111d292ca 100644 --- a/Services/DataX.Flow/Flow.ManagementService/Controllers/FlowManagementControllerDelete.cs +++ b/Services/DataX.Flow/Flow.ManagementService/Controllers/FlowManagementControllerDelete.cs @@ -12,7 +12,8 @@ using System.Collections.Generic; using System.Linq; using System.Threading.Tasks; - +using Microsoft.Extensions.Configuration; + namespace Flow.Management.Controllers { public partial class FlowManagementController : Controller @@ -26,7 +27,7 @@ public async Task DeleteFlow([FromBody]JObject queryObject) RolesCheck.EnsureWriter(Request); Ensure.NotNull(queryObject, "queryObject"); - ConfigDeleter c = new ConfigDeleter(_logger); + ConfigDeleter c = new ConfigDeleter(_logger, _configuration); return await c.DeleteFlow(queryObject); } catch (Exception e) diff --git a/Services/DataX.Flow/Flow.ManagementService/Flow.ManagementService.csproj b/Services/DataX.Flow/Flow.ManagementService/Flow.ManagementService.csproj index 2da7d3433..f8ee3b282 100644 --- a/Services/DataX.Flow/Flow.ManagementService/Flow.ManagementService.csproj +++ b/Services/DataX.Flow/Flow.ManagementService/Flow.ManagementService.csproj @@ -1,7 +1,7 @@ - + - netcoreapp2.1 + netcoreapp2.2 True win7-x64 False @@ -25,15 +25,14 @@ - - - + + all runtime; build; native; contentfiles; analyzers - - + + all @@ -43,16 +42,20 @@ + + + + @@ -62,14 +65,14 @@ - Microsoft + Microsoft400 StrongName - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/Flow.ManagementService/FlowManagementService.cs b/Services/DataX.Flow/Flow.ManagementService/FlowManagementService.cs index da55290a4..b9571a52a 100644 --- a/Services/DataX.Flow/Flow.ManagementService/FlowManagementService.cs +++ b/Services/DataX.Flow/Flow.ManagementService/FlowManagementService.cs @@ -26,9 +26,13 @@ namespace Flow.Management /// internal sealed class FlowManagementService : StatelessService { - public FlowManagementService(StatelessServiceContext context) + private readonly IWebHostBuilder _webHostBuilder; + + public FlowManagementService(StatelessServiceContext context, IWebHostBuilder webHostBuilder) : base(context) - { } + { + _webHostBuilder = webHostBuilder; + } /// /// Optional override to create listeners (like tcp, http) for this service instance. @@ -43,15 +47,13 @@ protected override IEnumerable CreateServiceInstanceLis { ServiceEventSource.Current.ServiceMessage(serviceContext, $"Starting Kestrel on {url}"); - return new WebHostBuilder() - .UseKestrel() + + + return _webHostBuilder .ConfigureServices( services => services .AddSingleton(serviceContext) ) - .UseContentRoot(Directory.GetCurrentDirectory()) - .UseStartup() - .UseServiceFabricIntegration(listener, ServiceFabricIntegrationOptions.None) .UseUrls(url) .Build(); diff --git a/Services/DataX.Flow/Flow.ManagementService/FlowManagementServiceStartup.cs b/Services/DataX.Flow/Flow.ManagementService/FlowManagementServiceStartup.cs new file mode 100644 index 000000000..e26dae247 --- /dev/null +++ b/Services/DataX.Flow/Flow.ManagementService/FlowManagementServiceStartup.cs @@ -0,0 +1,150 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Config; +using DataX.Config.ConfigDataModel; +using DataX.Config.ConfigurationProviders; +using DataX.Config.Local; +using DataX.Config.PublicService; +using DataX.Contract.Settings; +using DataX.Flow.Scheduler; +using DataX.ServiceHost.AspNetCore.Startup; +using DataX.Utilities.Blob; +using Flow.Management; +using Microsoft.AspNetCore.Builder; +using Microsoft.AspNetCore.Hosting; +using Microsoft.AspNetCore.Mvc; +using Microsoft.Extensions.Configuration; +using Microsoft.Extensions.DependencyInjection; +using Microsoft.Extensions.Hosting; +using Microsoft.Extensions.Logging; +using System; +using System.Collections.Generic; +using System.IO; +using System.Linq; +using System.Reflection; +using System.Threading.Tasks; + +namespace Flow.ManagementService +{ + /// + /// StartupFilter for Flow.ManagementService + /// + public sealed class FlowManagementServiceStartup : DataXServiceStartup + { + private const string _MetricsHttpEndpointRelativeUri = "/api/data/upload"; + + public FlowManagementServiceStartup() { } + + public FlowManagementServiceStartup(DataXSettings settings) + : base(settings) { } + + /// + public override void ConfigureServices(IServiceCollection services) + { + base.ConfigureServices(services); + services.AddSingleton(); + + // Initialize the settings by getting the values from settings file + InitConfigSettings(); + + var loggerFactory = services.BuildServiceProvider().GetRequiredService(); + + // Export the Config dependencies + Type[] exportTypes = new Type[] { typeof(FlowOperation), typeof(RuntimeConfigGeneration), typeof(JobOperation) }; + + IEnumerable dependencyAssemblies = Settings.EnableOneBox ? OneBoxModeDependencyAssemblies : CloudModeDependencyAssemblies; + IEnumerable additionalAssemblies = GetDependencyAssembliesFromStorageAsync().Result; + + var allAssemblies = dependencyAssemblies.Union(additionalAssemblies); + + services.AddMefExportsFromAssemblies(ServiceLifetime.Scoped, allAssemblies, exportTypes, null, loggerFactory, Settings.EnableOneBox); + } + + // Get all the dependencies needed to fulfill the ConfigGen + // requirements for cloud mode + private IList CloudModeDependencyAssemblies + => new List() + { + typeof(DataX.Config.ConfigGenConfiguration).Assembly, + typeof(DataX.Config.ConfigurationProviders.CosmosDbConfigurationProvider).Assembly, + typeof(DataX.Config.Storage.CosmosDBConfigStorage).Assembly, + typeof(DataX.Config.KeyVault.KeyVaultClient).Assembly, + typeof(DataX.Config.Input.EventHub.Processor.CreateEventHubConsumerGroup).Assembly, + Settings.SparkType == DataX.Config.ConfigDataModel.Constants.SparkTypeDataBricks ? typeof(DataX.Config.DatabricksClient.DatabricksClientFactory).Assembly : typeof(DataX.Config.LivyClient.LivyClientFactory).Assembly + }; + + // Get all the dependencies needed to fulfill the ConfigGen + // requirements for oneBox mode + private IList OneBoxModeDependencyAssemblies + => new List() + { + typeof(DataX.Config.ConfigGenConfiguration).Assembly, + typeof(DataX.Config.Local.LocalDesignTimeStorage).Assembly + }; + + // Get the required settings to bootstrap the config gen + private void InitConfigSettings() + { + InitialConfiguration.Set(Constants.ConfigSettingName_EnableOneBox, Settings.EnableOneBox.ToString()); + + if (!Settings.EnableOneBox) + { + InitialConfiguration.Set(CosmosDbConfigurationProvider.ConfigSettingName_CosmosDBConfig_ConnectionString, Settings.CosmosDBConfigConnectionString); + InitialConfiguration.Set(CosmosDbConfigurationProvider.ConfigSettingName_CosmosDBConfig_DatabaseName, Settings.CosmosDBConfigDatabaseName); + InitialConfiguration.Set(CosmosDbConfigurationProvider.ConfigSettingName_CosmosDBConfig_CollectionName, Settings.CosmosDBConfigCollectionName); + InitialConfiguration.Set(Constants.ConfigSettingName_ServiceKeyVaultName, Settings.ServiceKeyVaultName); + } + else + { + // Local settings + var metricsHttpEndpoint = Settings.MetricsHttpEndpoint.TrimEnd('/') + _MetricsHttpEndpointRelativeUri; + InitialConfiguration.Set(Constants.ConfigSettingName_LocalRoot, Settings.LocalRoot); + InitialConfiguration.Set(LocalSparkClient.ConfigSettingName_SparkHomeFolder, Settings.SparkHome); + InitialConfiguration.Set(Constants.ConfigSettingName_ClusterName, "localCluster"); + InitialConfiguration.Set(Constants.ConfigSettingName_ServiceKeyVaultName, "local"); + InitialConfiguration.Set(Constants.ConfigSettingName_RuntimeKeyVaultName, "local"); + InitialConfiguration.Set(Constants.ConfigSettingName_MetricEventHubConnectionKey, "local"); + InitialConfiguration.Set(Constants.ConfigSettingName_ConfigFolderContainerPath, ""); + InitialConfiguration.Set(Constants.ConfigSettingName_ConfigFolderHost, new Uri(Environment.CurrentDirectory).AbsoluteUri); + InitialConfiguration.Set(Constants.ConfigSettingName_LocalMetricsHttpEndpoint, metricsHttpEndpoint); + } + + } + + // Get additional assemblies from azure storage + private async Task> GetDependencyAssembliesFromStorageAsync() + { + IEnumerable additionalAssemblies = new List(); + var mefStorageAccountName = Settings.MefStorageAccountName; + var mefContainerName = Settings.MefContainerName; + + if (string.IsNullOrEmpty(mefStorageAccountName) || string.IsNullOrEmpty(mefContainerName)) + { + return additionalAssemblies; + } + + var mefBlobDirectory = Settings.MefBlobDirectory; + + BlobStorageMSI blobStorage = new BlobStorageMSI(mefStorageAccountName); + + var dlls = blobStorage.GetCloudBlockBlobs(mefContainerName, mefBlobDirectory); + + foreach (var blob in dlls) + { + if (blob.Name.EndsWith(".dll")) + { + using (var strm = new MemoryStream()) + { + await blob.DownloadToStreamAsync(strm); + byte[] asseblyBytes = strm.ToArray(); + var assembly = Assembly.Load(asseblyBytes); + additionalAssemblies = additionalAssemblies.Append(assembly); + } + } + } + return additionalAssemblies; + } + } +} diff --git a/Services/DataX.Flow/Flow.ManagementService/PackageRoot/Config/Settings.xml b/Services/DataX.Flow/Flow.ManagementService/PackageRoot/Config/Settings.xml index ffee6b2d3..d9d9ea570 100644 --- a/Services/DataX.Flow/Flow.ManagementService/PackageRoot/Config/Settings.xml +++ b/Services/DataX.Flow/Flow.ManagementService/PackageRoot/Config/Settings.xml @@ -11,8 +11,11 @@ Licensed under the MIT License + + + diff --git a/Services/DataX.Flow/Flow.ManagementService/Program.cs b/Services/DataX.Flow/Flow.ManagementService/Program.cs index 3d0376ca2..a77460a43 100644 --- a/Services/DataX.Flow/Flow.ManagementService/Program.cs +++ b/Services/DataX.Flow/Flow.ManagementService/Program.cs @@ -2,11 +2,15 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using DataX.ServiceHost; +using DataX.ServiceHost.AspNetCore.Extensions; +using Flow.ManagementService; +using Microsoft.AspNetCore.Hosting; using Microsoft.ServiceFabric.Services.Runtime; using System; using System.Diagnostics; +using System.IO; using System.Threading; -using System.Threading.Tasks; namespace Flow.Management { @@ -19,18 +23,28 @@ private static void Main() { try { - // The ServiceManifest.XML file defines one or more service type names. - // Registering a service maps a service type name to a .NET type. - // When Service Fabric creates an instance of this service type, - // an instance of the class is created in this host process. + if (HostUtil.InServiceFabric) + { + // The ServiceManifest.XML file defines one or more service type names. + // Registering a service maps a service type name to a .NET type. + // When Service Fabric creates an instance of this service type, + // an instance of the class is created in this host process. - ServiceRuntime.RegisterServiceAsync("Flow.ManagementServiceType", - context => new FlowManagementService(context)).GetAwaiter().GetResult(); + ServiceRuntime.RegisterServiceAsync("Flow.ManagementServiceType", + context => new FlowManagementService(context, WebHostBuilder)).GetAwaiter().GetResult(); - ServiceEventSource.Current.ServiceTypeRegistered(Process.GetCurrentProcess().Id, typeof(FlowManagementService).Name); + ServiceEventSource.Current.ServiceTypeRegistered(Process.GetCurrentProcess().Id, typeof(FlowManagementService).Name); - // Prevents this host process from terminating so services keeps running. - Thread.Sleep(Timeout.Infinite); + // Prevents this host process from terminating so services keeps running. + Thread.Sleep(Timeout.Infinite); + } + else + { + var webHost = WebHostBuilder.Build(); + + webHost.Start(); + webHost.WaitForShutdown(); + } } catch (Exception e) { @@ -38,5 +52,13 @@ private static void Main() throw; } } + + /// + /// Create a new WebHostBuilder with DataX default configuration that allows this to be run standalone + /// + private static IWebHostBuilder WebHostBuilder + => new WebHostBuilder() + .UseDataXDefaultConfiguration() + .Configure(app => app.UseDataXApplicationDefaults()); } } diff --git a/Services/DataX.Flow/Flow.ManagementService/Properties/launchSettings.json b/Services/DataX.Flow/Flow.ManagementService/Properties/launchSettings.json index 7c79bbd09..24a96075d 100644 --- a/Services/DataX.Flow/Flow.ManagementService/Properties/launchSettings.json +++ b/Services/DataX.Flow/Flow.ManagementService/Properties/launchSettings.json @@ -22,6 +22,7 @@ "launchBrowser": true, "launchUrl": "api/values", "environmentVariables": { + "AzureServicesAuthConnectionString": "RunAs=Developer; DeveloperTool=VisualStudio", "ASPNETCORE_ENVIRONMENT": "Development" }, "applicationUrl": "http://localhost:5000" diff --git a/Services/DataX.Flow/Flow.ManagementService/ServiceCollectionExtensions.cs b/Services/DataX.Flow/Flow.ManagementService/ServiceCollectionExtensions.cs index 004d10f6d..7ced7d1df 100644 --- a/Services/DataX.Flow/Flow.ManagementService/ServiceCollectionExtensions.cs +++ b/Services/DataX.Flow/Flow.ManagementService/ServiceCollectionExtensions.cs @@ -8,6 +8,8 @@ using System.Collections.Generic; using System.Composition.Hosting; using System.Reflection; +using DataX.Contract; +using Microsoft.Extensions.Logging; namespace Flow.Management { @@ -23,9 +25,9 @@ public static class ServiceCollectionExtensions /// Types being exported /// Instances that are created outside that need to be added to the container /// - public static IServiceCollection AddMefExportsFromAssemblies(this IServiceCollection services, ServiceLifetime lifetime, IEnumerable assemblies, Type[] exportTypes, object[] instanceExports) + public static IServiceCollection AddMefExportsFromAssemblies(this IServiceCollection services, ServiceLifetime lifetime, IEnumerable assemblies, Type[] exportTypes, object[] instanceExports, ILoggerFactory loggerFactory = null, bool local = false) { - var configuration = new ContainerConfiguration().WithAssemblies(assemblies).WithProvider(new InstanceExportDescriptorProvider(instanceExports)); + var configuration = new ContainerConfiguration().WithAssemblies(assemblies).WithProvider(new LoggerAndInstanceExportDescriptorProvider(instanceExports, loggerFactory)); using (var container = configuration.CreateContainer()) { foreach (var exportType in exportTypes) @@ -37,6 +39,14 @@ public static IServiceCollection AddMefExportsFromAssemblies(this IServiceCollec } } + if (local) + { + var localTemplateInitializer = container.GetExport(); + var result = localTemplateInitializer?.Initialize().Result; + + Ensure.IsSuccessResult(result); + } + } return services; } diff --git a/Services/DataX.Flow/Flow.ManagementService/Startup.cs b/Services/DataX.Flow/Flow.ManagementService/Startup.cs deleted file mode 100644 index a7e179b8b..000000000 --- a/Services/DataX.Flow/Flow.ManagementService/Startup.cs +++ /dev/null @@ -1,168 +0,0 @@ -// ********************************************************************* -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License -// ********************************************************************* -using DataX.Config; -using DataX.Config.ConfigurationProviders; -using DataX.Config.PublicService; -using DataX.Config.Storage; -using DataX.ServiceHost.ServiceFabric; -using DataX.Utilities.Blob; -using DataX.Utilities.Telemetry; -using Microsoft.AspNetCore.Builder; -using Microsoft.AspNetCore.Hosting; -using Microsoft.AspNetCore.Mvc; -using Microsoft.Extensions.Configuration; -using Microsoft.Extensions.DependencyInjection; -using Microsoft.Extensions.Logging; -using System; -using System.Collections.Generic; -using System.IO; -using System.Linq; -using System.Reflection; -using System.Threading.Tasks; - -namespace Flow.Management -{ - public class Startup - { - private readonly ILoggerFactory _loggerFactory; - public IConfiguration Configuration { get; } - private string _serviceKeyVaultName; - - public Startup(IHostingEnvironment env, ILoggerFactory loggerFactory) - { - var builder = new ConfigurationBuilder() - .SetBasePath(env.ContentRootPath) - .AddJsonFile("appsettings.json", optional: false, reloadOnChange: true) - .AddJsonFile($"appsettings.{env.EnvironmentName}.json", optional: true) - .AddEnvironmentVariables(); - - Configuration = builder.Build(); - - _loggerFactory = loggerFactory; - } - - // This method gets called by the runtime. Use this method to add services to the container. - public void ConfigureServices(IServiceCollection services) - { - services.AddMvc().SetCompatibilityVersion(CompatibilityVersion.Version_2_1); - _serviceKeyVaultName = ServiceFabricUtil.GetServiceKeyVaultName().Result.ToString(); - StartUpUtil.ConfigureServices(services); - - // Configure and create a logger instance to add it to MEF container - var logger = _loggerFactory.CreateLogger(); - - // Initialize the settings by getting the values from settings file - InitConfigSettings(); - - // Export the Config dependencies - Type[] exportTypes = new Type[] { typeof(FlowOperation), typeof(RuntimeConfigGeneration), typeof(JobOperation) }; - - IEnumerable cloudModeDependencyAssemblies = GetCloudModeDependencyAssemblies(); - IEnumerable additionalAssemblies = GetDependencyAssembliesFromStorageAsync().Result; - - var allAssemblies = cloudModeDependencyAssemblies.Union(additionalAssemblies); - - services.AddMefExportsFromAssemblies(ServiceLifetime.Scoped, allAssemblies, exportTypes, new object[] { logger }); - } - - // This method gets called by the runtime. Use this method to configure the HTTP request pipeline. - public void Configure(IApplicationBuilder app, IHostingEnvironment env, ILoggerFactory loggerFactory) - { - if (env.IsDevelopment()) - { - app.UseDeveloperExceptionPage(); - } - - // Set content-type options header to honor the server's mimetype - app.Use(async (context, next) => - { - context.Response.Headers.Add("X-Content-Type-Options", new string[] { "nosniff" }); - await next(); - }); - - // Configure logger that will be injected into the controller - app.UseMvc(); - } - - // Get all the dependencies needed to fulfill the ConfigGen - // requirements for cloud mode - private IList GetCloudModeDependencyAssemblies() - { - return new List() - { - typeof(DataX.Config.ConfigGenConfiguration).Assembly, - typeof(DataX.Config.ConfigurationProviders.CosmosDbConfigurationProvider).Assembly, - typeof(DataX.Config.Storage.CosmosDBConfigStorage).Assembly, - typeof(DataX.Config.KeyVault.KeyVaultClient).Assembly, - typeof(DataX.Config.LivyClient.LivyClientFactory).Assembly, - typeof(DataX.Config.Input.EventHub.Processor.CreateEventHubConsumerGroup).Assembly - }; - } - - // Get all the dependencies needed to fulfill the ConfigGen - // requirements for oneBox mode - private IList GetOneBoxModeDependencyAssemblies() - { - throw new NotImplementedException(); - } - - // Get the required settings to bootstrap the config gen - private void InitConfigSettings() - { - var cosmosDBConfigConnectionString = ServiceFabricUtil.GetServiceFabricConfigSetting("cosmosDBConfigConnectionString").Result.ToString(); - var cosmosDBConfigDatabaseName = ServiceFabricUtil.GetServiceFabricConfigSetting("cosmosDBConfigDatabaseName").Result.ToString(); - var cosmosDBConfigCollectionName = ServiceFabricUtil.GetServiceFabricConfigSetting("cosmosDBConfigCollectionName").Result.ToString(); - - InitialConfiguration.Set(CosmosDbConfigurationProvider.ConfigSettingName_CosmosDBConfig_ConnectionString, cosmosDBConfigConnectionString); - InitialConfiguration.Set(CosmosDbConfigurationProvider.ConfigSettingName_CosmosDBConfig_DatabaseName, cosmosDBConfigDatabaseName); - InitialConfiguration.Set(CosmosDbConfigurationProvider.ConfigSettingName_CosmosDBConfig_CollectionName, cosmosDBConfigCollectionName); - InitialConfiguration.Set(DataX.Config.ConfigDataModel.Constants.ConfigSettingName_ServiceKeyVaultName, _serviceKeyVaultName); - } - - // Get additional assemblies from azure storage - private async Task> GetDependencyAssembliesFromStorageAsync() - { - IEnumerable additionalAssemblies = new List(); - var mefStorageAccountName = ServiceFabricUtil.GetServiceFabricConfigSetting("MefStorageAccountName").Result.ToString(); - var mefContainerName = ServiceFabricUtil.GetServiceFabricConfigSetting("MefContainerName").Result.ToString(); - if (string.IsNullOrEmpty(mefStorageAccountName) || string.IsNullOrEmpty(mefContainerName)) - { - return additionalAssemblies; - } - - var mefBlobDirectory = ServiceFabricUtil.GetServiceFabricConfigSetting("MefBlobDirectory").Result.ToString(); - - BlobStorageMSI blobStorage = new BlobStorageMSI(mefStorageAccountName); - - var dlls = blobStorage.GetCloudBlockBlobs(mefContainerName, mefBlobDirectory); - - // Configure and create a logger instance - var logger = _loggerFactory.CreateLogger(); - - foreach (var blob in dlls) - { - if (blob.Name.EndsWith(".dll")) - { - using (var strm = new MemoryStream()) - { - await blob.DownloadToStreamAsync(strm); - byte[] asseblyBytes = strm.ToArray(); - try - { - var assembly = Assembly.Load(asseblyBytes); - additionalAssemblies = additionalAssemblies.Append(assembly); - } - catch(BadImageFormatException be) - { - // Do nothing and skip the assembly to load as it might be a native assembly - logger.LogError(be, "Unable to load Assembly: {0} from the StorageAccount", blob.Name); - } - } - } - } - return additionalAssemblies; - } - } -} diff --git a/Services/DataX.Flow/Flow.ManagementService/appsettings.Development.json b/Services/DataX.Flow/Flow.ManagementService/appsettings.Development.json index e203e9407..abf799d26 100644 --- a/Services/DataX.Flow/Flow.ManagementService/appsettings.Development.json +++ b/Services/DataX.Flow/Flow.ManagementService/appsettings.Development.json @@ -5,5 +5,37 @@ "System": "Information", "Microsoft": "Information" } + }, + "Kestrel": { + "EndPoints": { + "Http": { + "Url": "http://localhost:5000" + }, + "HttpsDefaultCert": { + "Url": "https://localhost:5001", + "Protocols": "Http1AndHttp2" + } + } + }, + "DataX": { + "ServiceEnvironment": { + "EnableOneBox": false, + "AppInsightsIntrumentationKey": "", + "CosmosDBConfigCollectionName": "", + "CosmosDBConfigConnectionString": "", + "CosmosDBConfigDatabaseName": "", + "ServiceKeyVaultName": "", + "TestClientId": "", + "MefBlobDirectory": null, + "MefContainerName": null, + "MefStorageAccountName": null, + "LocalRoot": "", + "SparkHome": "", + "MetricsHttpEndpoint": "http://localhost:2020/" + } + }, + "JwtBearerOptions": { + "Audience": "", + "Authority": null // JWT ISS (issuer) } -} +} \ No newline at end of file diff --git a/Services/DataX.Flow/Flow.ManagementService/appsettings.json b/Services/DataX.Flow/Flow.ManagementService/appsettings.json index 7c1d3b0db..9bf5600cf 100644 --- a/Services/DataX.Flow/Flow.ManagementService/appsettings.json +++ b/Services/DataX.Flow/Flow.ManagementService/appsettings.json @@ -4,5 +4,26 @@ "Default": "Information" } }, - "AllowedHosts": "*" + "AllowedHosts": "*", + "DataX": { + "ServiceEnvironment": { + "EnableOneBox": false, + "AppInsightsIntrumentationKey": "", + "CosmosDBConfigCollectionName": "", + "CosmosDBConfigConnectionString": "", + "CosmosDBConfigDatabaseName": "", + "MefBlobDirectory": null, + "MefContainerName": null, + "MefStorageAccountName": null, + "ServiceKeyVaultName": "", + "TestClientId": "", + "LocalRoot": "", + "SparkHome": "", + "MetricsHttpEndpoint": "http://localhost:2020/" + } + }, + "JwtBearerOptions": { + "Audience": "", + "Authority": null // JWT ISS (issuer) + } } \ No newline at end of file diff --git a/Services/DataX.Flow/Flow.SchemaInferenceService/Controllers/SchemaInferenceController.cs b/Services/DataX.Flow/Flow.SchemaInferenceService/Controllers/SchemaInferenceController.cs index fbac8b5c6..955b13a78 100644 --- a/Services/DataX.Flow/Flow.SchemaInferenceService/Controllers/SchemaInferenceController.cs +++ b/Services/DataX.Flow/Flow.SchemaInferenceService/Controllers/SchemaInferenceController.cs @@ -10,16 +10,24 @@ using System; using System.Threading.Tasks; using DataX.Utilities.Web; - +using DataX.ServiceHost.AspNetCore.Authorization.Roles; +using Microsoft.Extensions.Configuration; +using Microsoft.Extensions.Options; +using DataX.Contract.Settings; + namespace Flow.SchemaInferenceService.Controllers { [Route("api")] + [DataXWriter] public class SchemaInferenceController : Controller { private readonly ILogger _logger; - public SchemaInferenceController(ILogger logger) + private readonly IConfiguration _configuration; + public SchemaInferenceController(ILogger logger, IConfiguration configuration) { _logger = logger; + _configuration = configuration; + } [HttpPost] @@ -29,7 +37,7 @@ public async Task GetInputSchema([FromBody]JObject jObject) try { RolesCheck.EnsureWriter(Request); - SchemaInferenceManager sim = new SchemaInferenceManager(_logger); + SchemaInferenceManager sim = new SchemaInferenceManager(_logger, _configuration); return await sim.GetInputSchema(jObject); } @@ -47,7 +55,7 @@ public async Task RefreshSample([FromBody]JObject jObject) try { RolesCheck.EnsureWriter(Request); - SchemaInferenceManager sim = new SchemaInferenceManager(_logger); + SchemaInferenceManager sim = new SchemaInferenceManager(_logger, _configuration); return await sim.RefreshSample(jObject); } diff --git a/Services/DataX.Flow/Flow.SchemaInferenceService/Flow.SchemaInferenceService.csproj b/Services/DataX.Flow/Flow.SchemaInferenceService/Flow.SchemaInferenceService.csproj index 174dd426c..f35cee20c 100644 --- a/Services/DataX.Flow/Flow.SchemaInferenceService/Flow.SchemaInferenceService.csproj +++ b/Services/DataX.Flow/Flow.SchemaInferenceService/Flow.SchemaInferenceService.csproj @@ -1,7 +1,7 @@ - + - netcoreapp2.1 + netcoreapp2.2 True win7-x64 False @@ -25,15 +25,14 @@ - - - + + all runtime; build; native; contentfiles; analyzers - - + + all @@ -54,14 +53,14 @@ - Microsoft + Microsoft400 StrongName - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Flow/Flow.SchemaInferenceService/FlowSchemaInferenceServiceStartup.cs b/Services/DataX.Flow/Flow.SchemaInferenceService/FlowSchemaInferenceServiceStartup.cs new file mode 100644 index 000000000..852bd1898 --- /dev/null +++ b/Services/DataX.Flow/Flow.SchemaInferenceService/FlowSchemaInferenceServiceStartup.cs @@ -0,0 +1,24 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +namespace Flow.SchemaInferenceService +{ + using Microsoft.AspNetCore.Hosting; + using DataX.ServiceHost.AspNetCore; + using DataX.ServiceHost.AspNetCore.Startup; + using DataX.Contract.Settings; + + /// + /// StartupFilter for Flow.SchemaInferenceService + /// + public class FlowSchemaInferenceServiceStartup : DataXServiceStartup + { + public FlowSchemaInferenceServiceStartup() { } + + public FlowSchemaInferenceServiceStartup(DataXSettings settings) + : base(settings) { } + } +} + + diff --git a/Services/DataX.Flow/Flow.SchemaInferenceService/PackageRoot/Config/Settings.xml b/Services/DataX.Flow/Flow.SchemaInferenceService/PackageRoot/Config/Settings.xml index b4d8ee23f..12da835a2 100644 --- a/Services/DataX.Flow/Flow.SchemaInferenceService/PackageRoot/Config/Settings.xml +++ b/Services/DataX.Flow/Flow.SchemaInferenceService/PackageRoot/Config/Settings.xml @@ -12,5 +12,8 @@ Licensed under the MIT License + + + diff --git a/Services/DataX.Flow/Flow.SchemaInferenceService/Program.cs b/Services/DataX.Flow/Flow.SchemaInferenceService/Program.cs index b3d55b259..a0cd2bc24 100644 --- a/Services/DataX.Flow/Flow.SchemaInferenceService/Program.cs +++ b/Services/DataX.Flow/Flow.SchemaInferenceService/Program.cs @@ -2,6 +2,9 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using DataX.ServiceHost; +using DataX.ServiceHost.AspNetCore.Extensions; +using Microsoft.AspNetCore.Hosting; using Microsoft.ServiceFabric.Services.Runtime; using System; using System.Diagnostics; @@ -19,18 +22,28 @@ private static void Main() { try { - // The ServiceManifest.XML file defines one or more service type names. - // Registering a service maps a service type name to a .NET type. - // When Service Fabric creates an instance of this service type, - // an instance of the class is created in this host process. + if (HostUtil.InServiceFabric) + { + // The ServiceManifest.XML file defines one or more service type names. + // Registering a service maps a service type name to a .NET type. + // When Service Fabric creates an instance of this service type, + // an instance of the class is created in this host process. - ServiceRuntime.RegisterServiceAsync("Flow.SchemaInferenceServiceType", - context => new SchemaInferenceService(context)).GetAwaiter().GetResult(); + ServiceRuntime.RegisterServiceAsync("Flow.SchemaInferenceServiceType", + context => new SchemaInferenceService(context, WebHostBuilder)).GetAwaiter().GetResult(); - ServiceEventSource.Current.ServiceTypeRegistered(Process.GetCurrentProcess().Id, typeof(SchemaInferenceService).Name); + ServiceEventSource.Current.ServiceTypeRegistered(Process.GetCurrentProcess().Id, typeof(SchemaInferenceService).Name); - // Prevents this host process from terminating so services keeps running. - Thread.Sleep(Timeout.Infinite); + // Prevents this host process from terminating so services keeps running. + Thread.Sleep(Timeout.Infinite); + } + else + { + var webHost = WebHostBuilder.Build(); + + webHost.Start(); + webHost.WaitForShutdown(); + } } catch (Exception e) { @@ -38,5 +51,13 @@ private static void Main() throw; } } + + /// + /// Create a new WebHostBuilder with DataX default configuration that allows this to be run standalone + /// + private static IWebHostBuilder WebHostBuilder + => new WebHostBuilder() + .UseDataXDefaultConfiguration() + .Configure(app => app.UseDataXApplicationDefaults()); } } diff --git a/Services/DataX.Flow/Flow.SchemaInferenceService/Properties/launchSettings.json b/Services/DataX.Flow/Flow.SchemaInferenceService/Properties/launchSettings.json index 94547cac0..ef846773c 100644 --- a/Services/DataX.Flow/Flow.SchemaInferenceService/Properties/launchSettings.json +++ b/Services/DataX.Flow/Flow.SchemaInferenceService/Properties/launchSettings.json @@ -1,30 +1,30 @@ -{ - "$schema": "http://json.schemastore.org/launchsettings.json", - "iisSettings": { - "windowsAuthentication": false, - "anonymousAuthentication": true, - "iisExpress": { - "applicationUrl": "http://localhost:65448", - "sslPort": 0 - } - }, - "profiles": { - "IIS Express": { - "commandName": "IISExpress", - "launchBrowser": true, - "launchUrl": "api/values", - "environmentVariables": { - "ASPNETCORE_ENVIRONMENT": "Development" - } - }, - "Flow.SchemaInferenceService": { - "commandName": "Project", - "launchBrowser": true, - "launchUrl": "api/values", - "applicationUrl": "http://localhost:5000", - "environmentVariables": { - "ASPNETCORE_ENVIRONMENT": "Development" - } - } - } +{ + "iisSettings": { + "windowsAuthentication": false, + "anonymousAuthentication": true, + "iisExpress": { + "applicationUrl": "http://localhost:65448", + "sslPort": 0 + } + }, + "$schema": "http://json.schemastore.org/launchsettings.json", + "profiles": { + "IIS Express": { + "commandName": "IISExpress", + "launchBrowser": true, + "launchUrl": "api/values", + "environmentVariables": { + "ASPNETCORE_ENVIRONMENT": "Development" + } + }, + "Flow.SchemaInferenceService": { + "commandName": "Project", + "launchBrowser": true, + "launchUrl": "api/values", + "environmentVariables": { + "ASPNETCORE_ENVIRONMENT": "Development" + }, + "applicationUrl": "http://localhost:5000" + } + } } \ No newline at end of file diff --git a/Services/DataX.Flow/Flow.SchemaInferenceService/SchemaInferenceService.cs b/Services/DataX.Flow/Flow.SchemaInferenceService/SchemaInferenceService.cs index 51397a1e4..0a0aa0983 100644 --- a/Services/DataX.Flow/Flow.SchemaInferenceService/SchemaInferenceService.cs +++ b/Services/DataX.Flow/Flow.SchemaInferenceService/SchemaInferenceService.cs @@ -26,9 +26,13 @@ namespace Flow.SchemaInferenceService /// internal sealed class SchemaInferenceService : StatelessService { - public SchemaInferenceService(StatelessServiceContext context) + private readonly IWebHostBuilder _hostBuilder; + + public SchemaInferenceService(StatelessServiceContext context, IWebHostBuilder hostBuilder) : base(context) - { } + { + _hostBuilder = hostBuilder; + } /// /// Optional override to create listeners (like tcp, http) for this service instance. @@ -43,14 +47,10 @@ protected override IEnumerable CreateServiceInstanceLis { ServiceEventSource.Current.ServiceMessage(serviceContext, $"Starting Kestrel on {url}"); - return new WebHostBuilder() - .UseKestrel() + return _hostBuilder .ConfigureServices( services => services .AddSingleton(serviceContext)) - .UseContentRoot(Directory.GetCurrentDirectory()) - .UseStartup() - .UseServiceFabricIntegration(listener, ServiceFabricIntegrationOptions.None) .UseUrls(url) .Build(); diff --git a/Services/DataX.Flow/Flow.SchemaInferenceService/appsettings.Development.json b/Services/DataX.Flow/Flow.SchemaInferenceService/appsettings.Development.json index e203e9407..bcb491cc8 100644 --- a/Services/DataX.Flow/Flow.SchemaInferenceService/appsettings.Development.json +++ b/Services/DataX.Flow/Flow.SchemaInferenceService/appsettings.Development.json @@ -5,5 +5,31 @@ "System": "Information", "Microsoft": "Information" } + }, + "Kestrel": { + "EndPoints": { + "Http": { + "Url": "http://localhost:5000" + }, + "HttpsDefaultCert": { + "Url": "https://localhost:5001", + "Protocols": "Http1AndHttp2" + } + } + }, + "DataX": { + "ServiceEnvironment": { + "EnableOneBox": false, + "AppInsightsIntrumentationKey": "", + "CosmosDBConfigCollectionName": "", + "CosmosDBConfigConnectionString": "", + "CosmosDBConfigDatabaseName": "", + "ServiceKeyVaultName": "", + "TestClientId": "" + } + }, + "JwtBearerOptions": { + "Audience": "", + "Authority": null // JWT ISS (issuer) } -} +} \ No newline at end of file diff --git a/Services/DataX.Flow/Flow.SchemaInferenceService/appsettings.json b/Services/DataX.Flow/Flow.SchemaInferenceService/appsettings.json index 6a845cfd4..553554b07 100644 --- a/Services/DataX.Flow/Flow.SchemaInferenceService/appsettings.json +++ b/Services/DataX.Flow/Flow.SchemaInferenceService/appsettings.json @@ -4,5 +4,20 @@ "Default": "Information" } }, - "AllowedHosts": "*" + "AllowedHosts": "*", + "DataX": { + "ServiceEnvironment": { + "EnableOneBox": false, + "AppInsightsIntrumentationKey": "", + "CosmosDBConfigCollectionName": "", + "CosmosDBConfigConnectionString": "", + "CosmosDBConfigDatabaseName": "", + "ServiceKeyVaultName": "", + "TestClientId": "" + } + }, + "JwtBearerOptions": { + "Audience": "", + "Authority": null // JWT ISS (issuer) + } } diff --git a/Services/DataX.Flow/Flow.nuspec b/Services/DataX.Flow/Flow.nuspec index a23bc0299..f496cb298 100644 --- a/Services/DataX.Flow/Flow.nuspec +++ b/Services/DataX.Flow/Flow.nuspec @@ -16,7 +16,7 @@ - + \ No newline at end of file diff --git a/Services/DataX.Gateway/DataX.Gateway.Api.Tests/DataX.Gateway.Api.Tests.csproj b/Services/DataX.Gateway/DataX.Gateway.Api.Tests/DataX.Gateway.Api.Tests.csproj index 939e83f5c..9247921fd 100644 --- a/Services/DataX.Gateway/DataX.Gateway.Api.Tests/DataX.Gateway.Api.Tests.csproj +++ b/Services/DataX.Gateway/DataX.Gateway.Api.Tests/DataX.Gateway.Api.Tests.csproj @@ -207,7 +207,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Gateway/DataX.Gateway.Api/App.config b/Services/DataX.Gateway/DataX.Gateway.Api/App.config index 349b5d14a..72f6898a4 100644 --- a/Services/DataX.Gateway/DataX.Gateway.Api/App.config +++ b/Services/DataX.Gateway/DataX.Gateway.Api/App.config @@ -1,66 +1,90 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/Services/DataX.Gateway/DataX.Gateway.Api/Controllers/GatewayController.cs b/Services/DataX.Gateway/DataX.Gateway.Api/Controllers/GatewayController.cs index 086f9fd51..57035c584 100644 --- a/Services/DataX.Gateway/DataX.Gateway.Api/Controllers/GatewayController.cs +++ b/Services/DataX.Gateway/DataX.Gateway.Api/Controllers/GatewayController.cs @@ -41,6 +41,7 @@ public class GatewayController : ApiController private const int _DefaultHttpTimeoutSecs = Constants.DefaultHttpTimeoutSecs; private static readonly string _ReverseProxySslThumbprint; private static readonly string[] _AllowedUserRoles; + private static readonly HashSet _ClientWhitelist = new HashSet(); private static readonly ILogger _StaticLogger; private static readonly bool _IsUserInfoLoggingEnabled; @@ -87,8 +88,24 @@ static GatewayController() var configPackage = FabricRuntime.GetActivationContext().GetConfigurationPackageObject("Config"); var serviceEnvironmenConfig = configPackage.Settings.Sections["ServiceEnvironment"]; var appInsightsIntrumentationKey = serviceEnvironmenConfig.Parameters["AppInsightsIntrumentationKey"].Value; + var testClientId = serviceEnvironmenConfig.Parameters["TestClientId"].Value; _IsUserInfoLoggingEnabled = IsUserInfoLoggingEnabled(); _StaticLogger = new ApplicationInsightsLogger("GatewayILogger", new Microsoft.ApplicationInsights.TelemetryClient(new TelemetryConfiguration(KeyVault.GetSecretFromKeyvault(serviceKeyvaultName, appInsightsIntrumentationKey))), new ApplicationInsightsLoggerOptions()); + try + { + // Each secret needs to be a list of unique Ids in the format {ObjectId}.{TenantId} + List userIdList = KeyVault.GetSecretFromKeyvault(serviceKeyvaultName, testClientId).Split(new char[] { ',' }).ToList(); + foreach(string userId in userIdList) + { + _ClientWhitelist.Add(userId); + } + } + catch (Exception e) + { + // Do nothing in case the TestClientId is not set in the keyvault. This is set for testing purposes. + var message = e.Message; + _StaticLogger.LogError(e.Message); + } } private async Task Query(HttpRequestMessage request, string application, string service, string method, HttpMethod httpMethod) @@ -96,8 +113,9 @@ private async Task Query(HttpRequestMessage request, string applicati var roles = ((ClaimsIdentity)User.Identity).Claims .Where(c => c.Type == ClaimTypes.Role) .Select(c => c.Value).ToList(); - - if (roles.Intersect(_AllowedUserRoles).Any()) + var clientId = $"{ClaimsPrincipal.Current.FindFirst("http://schemas.microsoft.com/identity/claims/objectidentifier")?.Value}.{ClaimsPrincipal.Current.FindFirst("http://schemas.microsoft.com/identity/claims/tenantid")?.Value}"; + + if (roles.Intersect(_AllowedUserRoles).Any() || _ClientWhitelist.Contains(clientId)) { // Merge original headers with custom headers // Note: currently gets only the first value for a particular header @@ -153,9 +171,9 @@ public async Task QueryServiceWithPost(HttpRequestMessage request, st private Dictionary FetchUserHeaders() { var headers = new Dictionary(); - var userName = ClaimsPrincipal.Current.FindFirst(ClaimTypes.Upn) != null ? ClaimsPrincipal.Current.FindFirst(ClaimTypes.Upn).Value : (ClaimsPrincipal.Current.FindFirst(ClaimTypes.Email)?.Value); - headers.Add(_UserNameHeader, userName ?? string.Empty); - var userId = ClaimsPrincipal.Current.FindFirst("http://schemas.microsoft.com/identity/claims/objectidentifier")?.Value; + var userName = ClaimsPrincipal.Current.FindFirst(ClaimTypes.Upn) != null ? ClaimsPrincipal.Current.FindFirst(ClaimTypes.Upn).Value : ClaimsPrincipal.Current.FindFirst(ClaimTypes.Email)?.Value; + headers.Add(_UserNameHeader, userName ?? string.Empty); + var userId = $"{ClaimsPrincipal.Current.FindFirst("http://schemas.microsoft.com/identity/claims/objectidentifier")?.Value}.{ClaimsPrincipal.Current.FindFirst("http://schemas.microsoft.com/identity/claims/tenantid")?.Value}"; headers.Add(_UserIdHeader, userId ?? string.Empty); var roles = ((ClaimsIdentity)User.Identity).Claims.Where(c => c.Type == ClaimTypes.Role).Select(c => c.Value).ToList(); headers.Add(_UserRolesHeader, JArray.FromObject(roles).ToString(Formatting.None)); diff --git a/Services/DataX.Gateway/DataX.Gateway.Api/DataX.Gateway.Api.csproj b/Services/DataX.Gateway/DataX.Gateway.Api/DataX.Gateway.Api.csproj index f52956a7f..5cc9fd5b2 100644 --- a/Services/DataX.Gateway/DataX.Gateway.Api/DataX.Gateway.Api.csproj +++ b/Services/DataX.Gateway/DataX.Gateway.Api/DataX.Gateway.Api.csproj @@ -1,336 +1,340 @@ - - - - - - - - - - - Debug - x64 - {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD} - Exe - Properties - DataX.Gateway.Api - DataX.Gateway.Api - v4.6.1 - 512 - true - True - - - - - - true - full - false - bin\x64\Debug\ - DEBUG;TRACE - prompt - x64 - MinimumRecommendedRules.ruleset - - - pdbonly - true - bin\x64\Release\ - TRACE - prompt - x64 - MinimumRecommendedRules.ruleset - - - $(AdditionalFileItemNames);None - $(OutputPath) - - - true - - - FinalPublicKey.snk - - - True - - - true - - - - ..\packages\CommonServiceLocator.2.0.4\lib\net46\CommonServiceLocator.dll - - - ..\packages\Microsoft.ApplicationInsights.2.9.1\lib\net46\Microsoft.ApplicationInsights.dll - - - ..\packages\Microsoft.Azure.KeyVault.3.0.2\lib\net452\Microsoft.Azure.KeyVault.dll - - - ..\packages\Microsoft.Azure.KeyVault.WebKey.3.0.2\lib\net452\Microsoft.Azure.KeyVault.WebKey.dll - - - ..\packages\Microsoft.Azure.Services.AppAuthentication.1.0.3\lib\net452\Microsoft.Azure.Services.AppAuthentication.dll - - - - ..\packages\Microsoft.Extensions.Configuration.2.1.0\lib\netstandard2.0\Microsoft.Extensions.Configuration.dll - - - ..\packages\Microsoft.Extensions.Configuration.Abstractions.2.1.0\lib\netstandard2.0\Microsoft.Extensions.Configuration.Abstractions.dll - - - ..\packages\Microsoft.Extensions.Configuration.Binder.2.1.0\lib\netstandard2.0\Microsoft.Extensions.Configuration.Binder.dll - - - ..\packages\Microsoft.Extensions.DependencyInjection.Abstractions.2.1.0\lib\netstandard2.0\Microsoft.Extensions.DependencyInjection.Abstractions.dll - - - ..\packages\Microsoft.Extensions.Logging.2.1.0\lib\netstandard2.0\Microsoft.Extensions.Logging.dll - - - ..\packages\Microsoft.Extensions.Logging.Abstractions.2.1.0\lib\netstandard2.0\Microsoft.Extensions.Logging.Abstractions.dll - - - ..\packages\Microsoft.Extensions.Logging.ApplicationInsights.2.9.1\lib\netstandard2.0\Microsoft.Extensions.Logging.ApplicationInsights.dll - - - ..\packages\Microsoft.Extensions.Options.2.1.0\lib\netstandard2.0\Microsoft.Extensions.Options.dll - - - ..\packages\Microsoft.Extensions.Primitives.2.1.0\lib\netstandard2.0\Microsoft.Extensions.Primitives.dll - - - ..\packages\Microsoft.IdentityModel.Clients.ActiveDirectory.3.14.2\lib\net45\Microsoft.IdentityModel.Clients.ActiveDirectory.dll - - - ..\packages\Microsoft.IdentityModel.Clients.ActiveDirectory.3.14.2\lib\net45\Microsoft.IdentityModel.Clients.ActiveDirectory.Platform.dll - - - ..\packages\Microsoft.IdentityModel.JsonWebTokens.5.3.0\lib\net461\Microsoft.IdentityModel.JsonWebTokens.dll - - - ..\packages\Microsoft.IdentityModel.Logging.5.3.0\lib\net461\Microsoft.IdentityModel.Logging.dll - - - ..\packages\Microsoft.IdentityModel.Protocols.5.3.0\lib\net461\Microsoft.IdentityModel.Protocols.dll - - - ..\packages\Microsoft.IdentityModel.Protocols.WsFederation.5.3.0\lib\net461\Microsoft.IdentityModel.Protocols.WsFederation.dll - - - ..\packages\Microsoft.IdentityModel.Tokens.5.3.0\lib\net461\Microsoft.IdentityModel.Tokens.dll - - - ..\packages\Microsoft.IdentityModel.Tokens.Saml.5.3.0\lib\net461\Microsoft.IdentityModel.Tokens.Saml.dll - - - ..\packages\Microsoft.IdentityModel.Xml.5.3.0\lib\net461\Microsoft.IdentityModel.Xml.dll - - - ..\packages\Microsoft.Owin.4.0.0\lib\net451\Microsoft.Owin.dll - - - ..\packages\Microsoft.Owin.Host.HttpListener.4.0.0\lib\net451\Microsoft.Owin.Host.HttpListener.dll - - - ..\packages\Microsoft.Owin.Hosting.4.0.0\lib\net451\Microsoft.Owin.Hosting.dll - - - ..\packages\Microsoft.Owin.Security.4.0.0\lib\net451\Microsoft.Owin.Security.dll - - - ..\packages\Microsoft.Owin.Security.ActiveDirectory.4.0.0\lib\net451\Microsoft.Owin.Security.ActiveDirectory.dll - - - ..\packages\Microsoft.Owin.Security.Jwt.4.0.0\lib\net451\Microsoft.Owin.Security.Jwt.dll - - - ..\packages\Microsoft.Owin.Security.OAuth.4.0.0\lib\net451\Microsoft.Owin.Security.OAuth.dll - - - ..\packages\Microsoft.Rest.ClientRuntime.2.3.18\lib\net452\Microsoft.Rest.ClientRuntime.dll - - - ..\packages\Microsoft.Rest.ClientRuntime.Azure.3.3.18\lib\net452\Microsoft.Rest.ClientRuntime.Azure.dll - - - ..\packages\Microsoft.ServiceFabric.Data.3.3.617\lib\net461\Microsoft.ServiceFabric.Data.dll - - - ..\packages\Microsoft.ServiceFabric.Data.Extensions.1.4.617\lib\net461\Microsoft.ServiceFabric.Data.Extensions.dll - - - ..\packages\Microsoft.ServiceFabric.Data.Interfaces.3.3.617\lib\net461\Microsoft.ServiceFabric.Data.Interfaces.dll - - - ..\packages\Microsoft.ServiceFabric.Diagnostics.Internal.3.3.617\lib\net461\Microsoft.ServiceFabric.Diagnostics.dll - - - ..\packages\Microsoft.ServiceFabric.6.4.617\lib\net461\Microsoft.ServiceFabric.Internal.dll - - - ..\packages\Microsoft.ServiceFabric.6.4.617\lib\net461\Microsoft.ServiceFabric.Internal.Strings.dll - - - ..\packages\Microsoft.ServiceFabric.6.4.617\lib\net461\Microsoft.ServiceFabric.Preview.dll - - - ..\packages\Microsoft.ServiceFabric.Data.Extensions.1.4.617\lib\net461\Microsoft.ServiceFabric.ReliableCollection.Interop.dll - - - ..\packages\Microsoft.ServiceFabric.Services.3.3.617\lib\net461\Microsoft.ServiceFabric.Services.dll - - - ..\packages\Newtonsoft.Json.12.0.1\lib\net45\Newtonsoft.Json.dll - - - ..\packages\Owin.1.0\lib\net40\Owin.dll - True - - - - ..\packages\System.Buffers.4.5.0\lib\netstandard2.0\System.Buffers.dll - - - - ..\packages\System.Diagnostics.DiagnosticSource.4.5.1\lib\net46\System.Diagnostics.DiagnosticSource.dll - - - ..\packages\Microsoft.ServiceFabric.6.4.617\lib\net461\System.Fabric.dll - - - ..\packages\Microsoft.ServiceFabric.6.4.617\lib\net461\System.Fabric.Management.ServiceModel.dll - - - ..\packages\Microsoft.ServiceFabric.6.4.617\lib\net461\System.Fabric.Strings.dll - - - - ..\packages\System.IdentityModel.Tokens.Jwt.5.3.0\lib\net461\System.IdentityModel.Tokens.Jwt.dll - - - ..\packages\System.Memory.4.5.1\lib\netstandard2.0\System.Memory.dll - - - - - ..\packages\Microsoft.AspNet.WebApi.Client.5.2.7\lib\net45\System.Net.Http.Formatting.dll - - - - ..\packages\System.Net.Http.WinHttpHandler.4.5.1\lib\net461\System.Net.Http.WinHttpHandler.dll - - - - ..\packages\System.Numerics.Vectors.4.4.0\lib\net46\System.Numerics.Vectors.dll - - - - ..\packages\System.Runtime.CompilerServices.Unsafe.4.5.0\lib\netstandard2.0\System.Runtime.CompilerServices.Unsafe.dll - - - - - ..\packages\Microsoft.AspNet.WebApi.Core.5.2.7\lib\net45\System.Web.Http.dll - - - ..\packages\Microsoft.AspNet.WebApi.Owin.5.2.7\lib\net45\System.Web.Http.Owin.dll - - - - - - - - - - - - - - - - - - Designer - - - Designer - - - - Designer - - - - - - - - - {d16ba456-308e-407a-9617-22acb44d2052} - DataX.Contract - - - {ca675d17-c546-402f-8083-98feaa69db95} - DataX.ServiceHost.ServiceFabric - - - {F75FC501-DBD9-4990-92FE-193F34BEC268} - DataX.Utilities.KeyVault - - - {72f03d91-7dd7-4392-b25a-0790139e8774} - DataX.Gateway.Contract - - - - - - - This project references NuGet package(s) that are missing on this computer. Use NuGet Package Restore to download them. For more information, see http://go.microsoft.com/fwlink/?LinkID=322105. The missing file is {0}. - - - - - - - - - - - - - - - Microsoft - StrongName - - - - - Microsoft - StrongName - - - - - - - - - - - - - + + + + + + + + + + + Debug + x64 + {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD} + Exe + Properties + DataX.Gateway.Api + DataX.Gateway.Api + v4.6.1 + 512 + true + True + + + + + + true + full + false + bin\x64\Debug\ + DEBUG;TRACE + prompt + x64 + MinimumRecommendedRules.ruleset + + + pdbonly + true + bin\x64\Release\ + TRACE + prompt + x64 + MinimumRecommendedRules.ruleset + + + $(AdditionalFileItemNames);None + $(OutputPath) + + + true + + + FinalPublicKey.snk + + + True + + + true + + + + ..\packages\CommonServiceLocator.2.0.4\lib\net46\CommonServiceLocator.dll + + + ..\packages\Microsoft.ApplicationInsights.2.10.0\lib\net46\Microsoft.ApplicationInsights.dll + + + ..\packages\Microsoft.Azure.KeyVault.3.0.2\lib\net452\Microsoft.Azure.KeyVault.dll + + + ..\packages\Microsoft.Azure.KeyVault.WebKey.3.0.2\lib\net452\Microsoft.Azure.KeyVault.WebKey.dll + + + ..\packages\Microsoft.Azure.Services.AppAuthentication.1.0.3\lib\net452\Microsoft.Azure.Services.AppAuthentication.dll + + + + ..\packages\Microsoft.Extensions.Configuration.2.2.0\lib\netstandard2.0\Microsoft.Extensions.Configuration.dll + + + ..\packages\Microsoft.Extensions.Configuration.Abstractions.2.2.0\lib\netstandard2.0\Microsoft.Extensions.Configuration.Abstractions.dll + + + ..\packages\Microsoft.Extensions.Configuration.Binder.2.2.0\lib\netstandard2.0\Microsoft.Extensions.Configuration.Binder.dll + + + ..\packages\Microsoft.Extensions.DependencyInjection.Abstractions.2.2.0\lib\netstandard2.0\Microsoft.Extensions.DependencyInjection.Abstractions.dll + + + ..\packages\Microsoft.Extensions.Logging.2.2.0\lib\netstandard2.0\Microsoft.Extensions.Logging.dll + + + ..\packages\Microsoft.Extensions.Logging.Abstractions.2.2.0\lib\netstandard2.0\Microsoft.Extensions.Logging.Abstractions.dll + + + ..\packages\Microsoft.Extensions.Logging.ApplicationInsights.2.10.0\lib\netstandard2.0\Microsoft.Extensions.Logging.ApplicationInsights.dll + + + ..\packages\Microsoft.Extensions.Options.2.2.0\lib\netstandard2.0\Microsoft.Extensions.Options.dll + + + ..\packages\Microsoft.Extensions.Primitives.2.2.0\lib\netstandard2.0\Microsoft.Extensions.Primitives.dll + + + ..\packages\Microsoft.IdentityModel.Clients.ActiveDirectory.3.14.2\lib\net45\Microsoft.IdentityModel.Clients.ActiveDirectory.dll + + + ..\packages\Microsoft.IdentityModel.Clients.ActiveDirectory.3.14.2\lib\net45\Microsoft.IdentityModel.Clients.ActiveDirectory.Platform.dll + + + ..\packages\Microsoft.IdentityModel.JsonWebTokens.5.3.0\lib\net461\Microsoft.IdentityModel.JsonWebTokens.dll + + + ..\packages\Microsoft.IdentityModel.Logging.5.3.0\lib\net461\Microsoft.IdentityModel.Logging.dll + + + ..\packages\Microsoft.IdentityModel.Protocols.5.3.0\lib\net461\Microsoft.IdentityModel.Protocols.dll + + + ..\packages\Microsoft.IdentityModel.Protocols.WsFederation.5.3.0\lib\net461\Microsoft.IdentityModel.Protocols.WsFederation.dll + + + ..\packages\Microsoft.IdentityModel.Tokens.5.3.0\lib\net461\Microsoft.IdentityModel.Tokens.dll + + + ..\packages\Microsoft.IdentityModel.Tokens.Saml.5.3.0\lib\net461\Microsoft.IdentityModel.Tokens.Saml.dll + + + ..\packages\Microsoft.IdentityModel.Xml.5.3.0\lib\net461\Microsoft.IdentityModel.Xml.dll + + + ..\packages\Microsoft.Owin.4.0.0\lib\net451\Microsoft.Owin.dll + + + ..\packages\Microsoft.Owin.Host.HttpListener.4.0.0\lib\net451\Microsoft.Owin.Host.HttpListener.dll + + + ..\packages\Microsoft.Owin.Hosting.4.0.0\lib\net451\Microsoft.Owin.Hosting.dll + + + ..\packages\Microsoft.Owin.Security.4.0.0\lib\net451\Microsoft.Owin.Security.dll + + + ..\packages\Microsoft.Owin.Security.ActiveDirectory.4.0.0\lib\net451\Microsoft.Owin.Security.ActiveDirectory.dll + + + ..\packages\Microsoft.Owin.Security.Jwt.4.0.0\lib\net451\Microsoft.Owin.Security.Jwt.dll + + + ..\packages\Microsoft.Owin.Security.OAuth.4.0.0\lib\net451\Microsoft.Owin.Security.OAuth.dll + + + ..\packages\Microsoft.Rest.ClientRuntime.2.3.18\lib\net452\Microsoft.Rest.ClientRuntime.dll + + + ..\packages\Microsoft.Rest.ClientRuntime.Azure.3.3.18\lib\net452\Microsoft.Rest.ClientRuntime.Azure.dll + + + ..\packages\Microsoft.ServiceFabric.Data.3.3.617\lib\net461\Microsoft.ServiceFabric.Data.dll + + + ..\packages\Microsoft.ServiceFabric.Data.Extensions.1.4.617\lib\net461\Microsoft.ServiceFabric.Data.Extensions.dll + + + ..\packages\Microsoft.ServiceFabric.Data.Interfaces.3.3.617\lib\net461\Microsoft.ServiceFabric.Data.Interfaces.dll + + + ..\packages\Microsoft.ServiceFabric.Diagnostics.Internal.3.3.617\lib\net461\Microsoft.ServiceFabric.Diagnostics.dll + + + ..\packages\Microsoft.ServiceFabric.6.4.617\lib\net461\Microsoft.ServiceFabric.Internal.dll + + + ..\packages\Microsoft.ServiceFabric.6.4.617\lib\net461\Microsoft.ServiceFabric.Internal.Strings.dll + + + ..\packages\Microsoft.ServiceFabric.6.4.617\lib\net461\Microsoft.ServiceFabric.Preview.dll + + + ..\packages\Microsoft.ServiceFabric.Data.Extensions.1.4.617\lib\net461\Microsoft.ServiceFabric.ReliableCollection.Interop.dll + + + ..\packages\Microsoft.ServiceFabric.Services.3.3.617\lib\net461\Microsoft.ServiceFabric.Services.dll + + + ..\packages\Newtonsoft.Json.12.0.1\lib\net45\Newtonsoft.Json.dll + + + ..\packages\Owin.1.0\lib\net40\Owin.dll + True + + + + ..\packages\System.Buffers.4.5.0\lib\netstandard2.0\System.Buffers.dll + + + ..\packages\System.ComponentModel.Annotations.4.5.0\lib\net461\System.ComponentModel.Annotations.dll + + + + + ..\packages\System.Diagnostics.DiagnosticSource.4.5.1\lib\net46\System.Diagnostics.DiagnosticSource.dll + + + ..\packages\Microsoft.ServiceFabric.6.4.617\lib\net461\System.Fabric.dll + + + ..\packages\Microsoft.ServiceFabric.6.4.617\lib\net461\System.Fabric.Management.ServiceModel.dll + + + ..\packages\Microsoft.ServiceFabric.6.4.617\lib\net461\System.Fabric.Strings.dll + + + + ..\packages\System.IdentityModel.Tokens.Jwt.5.3.0\lib\net461\System.IdentityModel.Tokens.Jwt.dll + + + ..\packages\System.Memory.4.5.1\lib\netstandard2.0\System.Memory.dll + + + + + ..\packages\Microsoft.AspNet.WebApi.Client.5.2.7\lib\net45\System.Net.Http.Formatting.dll + + + + ..\packages\System.Net.Http.WinHttpHandler.4.5.1\lib\net461\System.Net.Http.WinHttpHandler.dll + + + + ..\packages\System.Numerics.Vectors.4.4.0\lib\net46\System.Numerics.Vectors.dll + + + + ..\packages\System.Runtime.CompilerServices.Unsafe.4.5.1\lib\netstandard2.0\System.Runtime.CompilerServices.Unsafe.dll + + + + + ..\packages\Microsoft.AspNet.WebApi.Core.5.2.7\lib\net45\System.Web.Http.dll + + + ..\packages\Microsoft.AspNet.WebApi.Owin.5.2.7\lib\net45\System.Web.Http.Owin.dll + + + + + + + + + + + + + + + + + + Designer + + + Designer + + + + Designer + + + + + + + + + {d16ba456-308e-407a-9617-22acb44d2052} + DataX.Contract + + + {ca675d17-c546-402f-8083-98feaa69db95} + DataX.ServiceHost.ServiceFabric + + + {F75FC501-DBD9-4990-92FE-193F34BEC268} + DataX.Utilities.KeyVault + + + {72f03d91-7dd7-4392-b25a-0790139e8774} + DataX.Gateway.Contract + + + + + + + This project references NuGet package(s) that are missing on this computer. Use NuGet Package Restore to download them. For more information, see http://go.microsoft.com/fwlink/?LinkID=322105. The missing file is {0}. + + + + + + + + + + + + + + + Microsoft400 + StrongName + + + + + Microsoft400 + StrongName + + + + + + + + + + + + + \ No newline at end of file diff --git a/Services/DataX.Gateway/DataX.Gateway.Api/PackageRoot/Config/Settings.xml b/Services/DataX.Gateway/DataX.Gateway.Api/PackageRoot/Config/Settings.xml index 8b7063309..4883de490 100644 --- a/Services/DataX.Gateway/DataX.Gateway.Api/PackageRoot/Config/Settings.xml +++ b/Services/DataX.Gateway/DataX.Gateway.Api/PackageRoot/Config/Settings.xml @@ -19,6 +19,7 @@ Licensed under the MIT License
+
diff --git a/Services/DataX.Gateway/DataX.Gateway.Api/packages.config b/Services/DataX.Gateway/DataX.Gateway.Api/packages.config index 229abd176..fc428de71 100644 --- a/Services/DataX.Gateway/DataX.Gateway.Api/packages.config +++ b/Services/DataX.Gateway/DataX.Gateway.Api/packages.config @@ -1,63 +1,64 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/Services/DataX.Gateway/DataX.Gateway.Contract/DataX.Gateway.Contract.csproj b/Services/DataX.Gateway/DataX.Gateway.Contract/DataX.Gateway.Contract.csproj index 24d148e0e..f642e4e5b 100644 --- a/Services/DataX.Gateway/DataX.Gateway.Contract/DataX.Gateway.Contract.csproj +++ b/Services/DataX.Gateway/DataX.Gateway.Contract/DataX.Gateway.Contract.csproj @@ -21,7 +21,7 @@ - Microsoft + Microsoft400 StrongName @@ -39,7 +39,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Gateway/DataX.Gateway.sln b/Services/DataX.Gateway/DataX.Gateway.sln index f63e0bb39..21fd497e9 100644 --- a/Services/DataX.Gateway/DataX.Gateway.sln +++ b/Services/DataX.Gateway/DataX.Gateway.sln @@ -1,97 +1,107 @@ - -Microsoft Visual Studio Solution File, Format Version 12.00 -# Visual Studio 15 -VisualStudioVersion = 15.0.28307.106 -MinimumVisualStudioVersion = 10.0.40219.1 -Project("{A07B5EB6-E848-4116-A8D0-A826331D98C6}") = "DataX.Gateway", "DataX.Gateway\DataX.Gateway.sfproj", "{1356403C-69C6-429B-AC77-38780284A658}" -EndProject -Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "DataX.Gateway.Api", "DataX.Gateway.Api\DataX.Gateway.Api.csproj", "{AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.KeyVault", "..\DataX.Utilities\DataX.Utilities.KeyVault\DataX.Utilities.KeyVault.csproj", "{F75FC501-DBD9-4990-92FE-193F34BEC268}" -EndProject -Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Solution Items", "Solution Items", "{C755BE8A-9B2F-4E10-B2C6-F0E6436822FB}" - ProjectSection(SolutionItems) = preProject - .editorconfig = .editorconfig - EndProjectSection -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Contract", "..\DataX.Contract\DataX.Contract.csproj", "{D16BA456-308E-407A-9617-22ACB44D2052}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.ServiceHost.ServiceFabric", "..\DataX.ServiceHost\DataX.ServiceHost.ServiceFabric\DataX.ServiceHost.ServiceFabric.csproj", "{CA675D17-C546-402F-8083-98FEAA69DB95}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Gateway.Contract", "DataX.Gateway.Contract\DataX.Gateway.Contract.csproj", "{72F03D91-7DD7-4392-B25A-0790139E8774}" -EndProject -Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "DataX.Utility.Nuget", "..\DataX.Utilities\DataX.Utility.Nuget\DataX.Utility.Nuget.csproj", "{7BE24BB5-3237-4403-921E-3CFC5A4A8E20}" -EndProject -Global - GlobalSection(SolutionConfigurationPlatforms) = preSolution - Debug|Any CPU = Debug|Any CPU - Debug|x64 = Debug|x64 - Release|Any CPU = Release|Any CPU - Release|x64 = Release|x64 - EndGlobalSection - GlobalSection(ProjectConfigurationPlatforms) = postSolution - {1356403C-69C6-429B-AC77-38780284A658}.Debug|Any CPU.ActiveCfg = Debug|x64 - {1356403C-69C6-429B-AC77-38780284A658}.Debug|Any CPU.Build.0 = Debug|x64 - {1356403C-69C6-429B-AC77-38780284A658}.Debug|Any CPU.Deploy.0 = Debug|x64 - {1356403C-69C6-429B-AC77-38780284A658}.Debug|x64.ActiveCfg = Debug|x64 - {1356403C-69C6-429B-AC77-38780284A658}.Debug|x64.Build.0 = Debug|x64 - {1356403C-69C6-429B-AC77-38780284A658}.Debug|x64.Deploy.0 = Debug|x64 - {1356403C-69C6-429B-AC77-38780284A658}.Release|Any CPU.ActiveCfg = Release|x64 - {1356403C-69C6-429B-AC77-38780284A658}.Release|x64.ActiveCfg = Release|x64 - {1356403C-69C6-429B-AC77-38780284A658}.Release|x64.Build.0 = Release|x64 - {1356403C-69C6-429B-AC77-38780284A658}.Release|x64.Deploy.0 = Release|x64 - {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Debug|Any CPU.ActiveCfg = Debug|x64 - {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Debug|Any CPU.Build.0 = Debug|x64 - {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Debug|x64.ActiveCfg = Debug|x64 - {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Debug|x64.Build.0 = Debug|x64 - {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Release|Any CPU.ActiveCfg = Release|x64 - {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Release|x64.ActiveCfg = Release|x64 - {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Release|x64.Build.0 = Release|x64 - {F75FC501-DBD9-4990-92FE-193F34BEC268}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {F75FC501-DBD9-4990-92FE-193F34BEC268}.Debug|Any CPU.Build.0 = Debug|Any CPU - {F75FC501-DBD9-4990-92FE-193F34BEC268}.Debug|x64.ActiveCfg = Debug|Any CPU - {F75FC501-DBD9-4990-92FE-193F34BEC268}.Debug|x64.Build.0 = Debug|Any CPU - {F75FC501-DBD9-4990-92FE-193F34BEC268}.Release|Any CPU.ActiveCfg = Release|Any CPU - {F75FC501-DBD9-4990-92FE-193F34BEC268}.Release|Any CPU.Build.0 = Release|Any CPU - {F75FC501-DBD9-4990-92FE-193F34BEC268}.Release|x64.ActiveCfg = Release|Any CPU - {F75FC501-DBD9-4990-92FE-193F34BEC268}.Release|x64.Build.0 = Release|Any CPU - {D16BA456-308E-407A-9617-22ACB44D2052}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {D16BA456-308E-407A-9617-22ACB44D2052}.Debug|Any CPU.Build.0 = Debug|Any CPU - {D16BA456-308E-407A-9617-22ACB44D2052}.Debug|x64.ActiveCfg = Debug|Any CPU - {D16BA456-308E-407A-9617-22ACB44D2052}.Debug|x64.Build.0 = Debug|Any CPU - {D16BA456-308E-407A-9617-22ACB44D2052}.Release|Any CPU.ActiveCfg = Release|Any CPU - {D16BA456-308E-407A-9617-22ACB44D2052}.Release|Any CPU.Build.0 = Release|Any CPU - {D16BA456-308E-407A-9617-22ACB44D2052}.Release|x64.ActiveCfg = Release|Any CPU - {D16BA456-308E-407A-9617-22ACB44D2052}.Release|x64.Build.0 = Release|Any CPU - {CA675D17-C546-402F-8083-98FEAA69DB95}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {CA675D17-C546-402F-8083-98FEAA69DB95}.Debug|Any CPU.Build.0 = Debug|Any CPU - {CA675D17-C546-402F-8083-98FEAA69DB95}.Debug|x64.ActiveCfg = Debug|Any CPU - {CA675D17-C546-402F-8083-98FEAA69DB95}.Debug|x64.Build.0 = Debug|Any CPU - {CA675D17-C546-402F-8083-98FEAA69DB95}.Release|Any CPU.ActiveCfg = Release|Any CPU - {CA675D17-C546-402F-8083-98FEAA69DB95}.Release|Any CPU.Build.0 = Release|Any CPU - {CA675D17-C546-402F-8083-98FEAA69DB95}.Release|x64.ActiveCfg = Release|Any CPU - {CA675D17-C546-402F-8083-98FEAA69DB95}.Release|x64.Build.0 = Release|Any CPU - {72F03D91-7DD7-4392-B25A-0790139E8774}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {72F03D91-7DD7-4392-B25A-0790139E8774}.Debug|Any CPU.Build.0 = Debug|Any CPU - {72F03D91-7DD7-4392-B25A-0790139E8774}.Debug|x64.ActiveCfg = Debug|Any CPU - {72F03D91-7DD7-4392-B25A-0790139E8774}.Debug|x64.Build.0 = Debug|Any CPU - {72F03D91-7DD7-4392-B25A-0790139E8774}.Release|Any CPU.ActiveCfg = Release|Any CPU - {72F03D91-7DD7-4392-B25A-0790139E8774}.Release|Any CPU.Build.0 = Release|Any CPU - {72F03D91-7DD7-4392-B25A-0790139E8774}.Release|x64.ActiveCfg = Release|Any CPU - {72F03D91-7DD7-4392-B25A-0790139E8774}.Release|x64.Build.0 = Release|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|Any CPU.Build.0 = Debug|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|x64.ActiveCfg = Debug|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|x64.Build.0 = Debug|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|Any CPU.ActiveCfg = Release|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|Any CPU.Build.0 = Release|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|x64.ActiveCfg = Release|Any CPU - {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|x64.Build.0 = Release|Any CPU - EndGlobalSection - GlobalSection(SolutionProperties) = preSolution - HideSolutionNode = FALSE - EndGlobalSection - GlobalSection(ExtensibilityGlobals) = postSolution - SolutionGuid = {808B015D-E92F-4010-84A6-AA93AEE20187} - EndGlobalSection -EndGlobal + +Microsoft Visual Studio Solution File, Format Version 12.00 +# Visual Studio 15 +VisualStudioVersion = 15.0.28307.106 +MinimumVisualStudioVersion = 10.0.40219.1 +Project("{A07B5EB6-E848-4116-A8D0-A826331D98C6}") = "DataX.Gateway", "DataX.Gateway\DataX.Gateway.sfproj", "{1356403C-69C6-429B-AC77-38780284A658}" +EndProject +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "DataX.Gateway.Api", "DataX.Gateway.Api\DataX.Gateway.Api.csproj", "{AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.KeyVault", "..\DataX.Utilities\DataX.Utilities.KeyVault\DataX.Utilities.KeyVault.csproj", "{F75FC501-DBD9-4990-92FE-193F34BEC268}" +EndProject +Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Solution Items", "Solution Items", "{C755BE8A-9B2F-4E10-B2C6-F0E6436822FB}" + ProjectSection(SolutionItems) = preProject + .editorconfig = .editorconfig + EndProjectSection +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Contract", "..\DataX.Contract\DataX.Contract.csproj", "{D16BA456-308E-407A-9617-22ACB44D2052}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.ServiceHost.ServiceFabric", "..\DataX.ServiceHost\DataX.ServiceHost.ServiceFabric\DataX.ServiceHost.ServiceFabric.csproj", "{CA675D17-C546-402F-8083-98FEAA69DB95}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Gateway.Contract", "DataX.Gateway.Contract\DataX.Gateway.Contract.csproj", "{72F03D91-7DD7-4392-B25A-0790139E8774}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.Nuget", "..\DataX.Utilities\DataX.Utility.Nuget\DataX.Utility.Nuget.csproj", "{7BE24BB5-3237-4403-921E-3CFC5A4A8E20}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.ServiceHost", "..\DataX.ServiceHost\DataX.ServiceHost\DataX.ServiceHost.csproj", "{C96A9A61-E881-46D6-BD29-239E2DE13FC1}" +EndProject +Global + GlobalSection(SolutionConfigurationPlatforms) = preSolution + Debug|Any CPU = Debug|Any CPU + Debug|x64 = Debug|x64 + Release|Any CPU = Release|Any CPU + Release|x64 = Release|x64 + EndGlobalSection + GlobalSection(ProjectConfigurationPlatforms) = postSolution + {1356403C-69C6-429B-AC77-38780284A658}.Debug|Any CPU.ActiveCfg = Debug|x64 + {1356403C-69C6-429B-AC77-38780284A658}.Debug|Any CPU.Build.0 = Debug|x64 + {1356403C-69C6-429B-AC77-38780284A658}.Debug|Any CPU.Deploy.0 = Debug|x64 + {1356403C-69C6-429B-AC77-38780284A658}.Debug|x64.ActiveCfg = Debug|x64 + {1356403C-69C6-429B-AC77-38780284A658}.Debug|x64.Build.0 = Debug|x64 + {1356403C-69C6-429B-AC77-38780284A658}.Debug|x64.Deploy.0 = Debug|x64 + {1356403C-69C6-429B-AC77-38780284A658}.Release|Any CPU.ActiveCfg = Release|x64 + {1356403C-69C6-429B-AC77-38780284A658}.Release|x64.ActiveCfg = Release|x64 + {1356403C-69C6-429B-AC77-38780284A658}.Release|x64.Build.0 = Release|x64 + {1356403C-69C6-429B-AC77-38780284A658}.Release|x64.Deploy.0 = Release|x64 + {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Debug|Any CPU.ActiveCfg = Debug|x64 + {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Debug|Any CPU.Build.0 = Debug|x64 + {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Debug|x64.ActiveCfg = Debug|x64 + {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Debug|x64.Build.0 = Debug|x64 + {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Release|Any CPU.ActiveCfg = Release|x64 + {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Release|x64.ActiveCfg = Release|x64 + {AC26DE00-E2C4-492F-98F4-A8BDD14F8EFD}.Release|x64.Build.0 = Release|x64 + {F75FC501-DBD9-4990-92FE-193F34BEC268}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {F75FC501-DBD9-4990-92FE-193F34BEC268}.Debug|Any CPU.Build.0 = Debug|Any CPU + {F75FC501-DBD9-4990-92FE-193F34BEC268}.Debug|x64.ActiveCfg = Debug|Any CPU + {F75FC501-DBD9-4990-92FE-193F34BEC268}.Debug|x64.Build.0 = Debug|Any CPU + {F75FC501-DBD9-4990-92FE-193F34BEC268}.Release|Any CPU.ActiveCfg = Release|Any CPU + {F75FC501-DBD9-4990-92FE-193F34BEC268}.Release|Any CPU.Build.0 = Release|Any CPU + {F75FC501-DBD9-4990-92FE-193F34BEC268}.Release|x64.ActiveCfg = Release|Any CPU + {F75FC501-DBD9-4990-92FE-193F34BEC268}.Release|x64.Build.0 = Release|Any CPU + {D16BA456-308E-407A-9617-22ACB44D2052}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {D16BA456-308E-407A-9617-22ACB44D2052}.Debug|Any CPU.Build.0 = Debug|Any CPU + {D16BA456-308E-407A-9617-22ACB44D2052}.Debug|x64.ActiveCfg = Debug|Any CPU + {D16BA456-308E-407A-9617-22ACB44D2052}.Debug|x64.Build.0 = Debug|Any CPU + {D16BA456-308E-407A-9617-22ACB44D2052}.Release|Any CPU.ActiveCfg = Release|Any CPU + {D16BA456-308E-407A-9617-22ACB44D2052}.Release|Any CPU.Build.0 = Release|Any CPU + {D16BA456-308E-407A-9617-22ACB44D2052}.Release|x64.ActiveCfg = Release|Any CPU + {D16BA456-308E-407A-9617-22ACB44D2052}.Release|x64.Build.0 = Release|Any CPU + {CA675D17-C546-402F-8083-98FEAA69DB95}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {CA675D17-C546-402F-8083-98FEAA69DB95}.Debug|Any CPU.Build.0 = Debug|Any CPU + {CA675D17-C546-402F-8083-98FEAA69DB95}.Debug|x64.ActiveCfg = Debug|Any CPU + {CA675D17-C546-402F-8083-98FEAA69DB95}.Debug|x64.Build.0 = Debug|Any CPU + {CA675D17-C546-402F-8083-98FEAA69DB95}.Release|Any CPU.ActiveCfg = Release|Any CPU + {CA675D17-C546-402F-8083-98FEAA69DB95}.Release|Any CPU.Build.0 = Release|Any CPU + {CA675D17-C546-402F-8083-98FEAA69DB95}.Release|x64.ActiveCfg = Release|Any CPU + {CA675D17-C546-402F-8083-98FEAA69DB95}.Release|x64.Build.0 = Release|Any CPU + {72F03D91-7DD7-4392-B25A-0790139E8774}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {72F03D91-7DD7-4392-B25A-0790139E8774}.Debug|Any CPU.Build.0 = Debug|Any CPU + {72F03D91-7DD7-4392-B25A-0790139E8774}.Debug|x64.ActiveCfg = Debug|Any CPU + {72F03D91-7DD7-4392-B25A-0790139E8774}.Debug|x64.Build.0 = Debug|Any CPU + {72F03D91-7DD7-4392-B25A-0790139E8774}.Release|Any CPU.ActiveCfg = Release|Any CPU + {72F03D91-7DD7-4392-B25A-0790139E8774}.Release|Any CPU.Build.0 = Release|Any CPU + {72F03D91-7DD7-4392-B25A-0790139E8774}.Release|x64.ActiveCfg = Release|Any CPU + {72F03D91-7DD7-4392-B25A-0790139E8774}.Release|x64.Build.0 = Release|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|Any CPU.Build.0 = Debug|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|x64.ActiveCfg = Debug|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Debug|x64.Build.0 = Debug|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|Any CPU.ActiveCfg = Release|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|Any CPU.Build.0 = Release|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|x64.ActiveCfg = Release|Any CPU + {7BE24BB5-3237-4403-921E-3CFC5A4A8E20}.Release|x64.Build.0 = Release|Any CPU + {C96A9A61-E881-46D6-BD29-239E2DE13FC1}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {C96A9A61-E881-46D6-BD29-239E2DE13FC1}.Debug|Any CPU.Build.0 = Debug|Any CPU + {C96A9A61-E881-46D6-BD29-239E2DE13FC1}.Debug|x64.ActiveCfg = Debug|Any CPU + {C96A9A61-E881-46D6-BD29-239E2DE13FC1}.Debug|x64.Build.0 = Debug|Any CPU + {C96A9A61-E881-46D6-BD29-239E2DE13FC1}.Release|Any CPU.ActiveCfg = Release|Any CPU + {C96A9A61-E881-46D6-BD29-239E2DE13FC1}.Release|Any CPU.Build.0 = Release|Any CPU + {C96A9A61-E881-46D6-BD29-239E2DE13FC1}.Release|x64.ActiveCfg = Release|Any CPU + {C96A9A61-E881-46D6-BD29-239E2DE13FC1}.Release|x64.Build.0 = Release|Any CPU + EndGlobalSection + GlobalSection(SolutionProperties) = preSolution + HideSolutionNode = FALSE + EndGlobalSection + GlobalSection(ExtensibilityGlobals) = postSolution + SolutionGuid = {808B015D-E92F-4010-84A6-AA93AEE20187} + EndGlobalSection +EndGlobal diff --git a/Services/DataX.Gateway/DataX.Gateway/ApplicationPackageRoot/ApplicationManifest.xml b/Services/DataX.Gateway/DataX.Gateway/ApplicationPackageRoot/ApplicationManifest.xml index ebe73159f..74b154d66 100644 --- a/Services/DataX.Gateway/DataX.Gateway/ApplicationPackageRoot/ApplicationManifest.xml +++ b/Services/DataX.Gateway/DataX.Gateway/ApplicationPackageRoot/ApplicationManifest.xml @@ -1,63 +1,65 @@ - - - - - - - - - - - - - - - - - - - - - - - - -
- - -
-
- -
-
- -
-
- - - -
-
-
-
- - - - - - -
- - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + +
+ + +
+
+ +
+
+ +
+
+ + + + +
+
+
+
+ + + + + + +
+ + + + + + + + + +
\ No newline at end of file diff --git a/Services/DataX.Gateway/DataX.Gateway/ApplicationParameters/Cloud.xml b/Services/DataX.Gateway/DataX.Gateway/ApplicationParameters/Cloud.xml index d85b05045..c5b1a0d26 100644 --- a/Services/DataX.Gateway/DataX.Gateway/ApplicationParameters/Cloud.xml +++ b/Services/DataX.Gateway/DataX.Gateway/ApplicationParameters/Cloud.xml @@ -1,23 +1,24 @@ - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/Services/DataX.Gateway/DataX.Gateway/ApplicationParameters/Local.1Node.xml b/Services/DataX.Gateway/DataX.Gateway/ApplicationParameters/Local.1Node.xml index 1ef8a3bce..adeabc9ba 100644 --- a/Services/DataX.Gateway/DataX.Gateway/ApplicationParameters/Local.1Node.xml +++ b/Services/DataX.Gateway/DataX.Gateway/ApplicationParameters/Local.1Node.xml @@ -18,6 +18,7 @@ Licensed under the MIT License + \ No newline at end of file diff --git a/Services/DataX.Gateway/DataX.Gateway/ApplicationParameters/Local.5Node.xml b/Services/DataX.Gateway/DataX.Gateway/ApplicationParameters/Local.5Node.xml index 761a8fad1..be914a788 100644 --- a/Services/DataX.Gateway/DataX.Gateway/ApplicationParameters/Local.5Node.xml +++ b/Services/DataX.Gateway/DataX.Gateway/ApplicationParameters/Local.5Node.xml @@ -1,23 +1,24 @@ - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/Services/DataX.Gateway/DataX.Gateway/DataX.Gateway.sfproj b/Services/DataX.Gateway/DataX.Gateway/DataX.Gateway.sfproj index 1ca41ea49..ba9855ebc 100644 --- a/Services/DataX.Gateway/DataX.Gateway/DataX.Gateway.sfproj +++ b/Services/DataX.Gateway/DataX.Gateway/DataX.Gateway.sfproj @@ -1,46 +1,46 @@ - - - - - 1356403c-69c6-429b-ac77-38780284a658 - 2.1 - 1.5 - 1.6.6 - - - - Debug - x64 - - - Release - x64 - - - - - - - - - - - - - - - - - - - - - $(MSBuildExtensionsPath32)\Microsoft\VisualStudio\v$(VisualStudioVersion)\Service Fabric Tools\Microsoft.VisualStudio.Azure.Fabric.ApplicationProject.targets - - - - - - - + + + + + 1356403c-69c6-429b-ac77-38780284a658 + 2.1 + 1.5 + 1.6.6 + + + + Debug + x64 + + + Release + x64 + + + + + + + + + + + + + + + + + + + + + $(MSBuildExtensionsPath32)\Microsoft\VisualStudio\v$(VisualStudioVersion)\Service Fabric Tools\Microsoft.VisualStudio.Azure.Fabric.ApplicationProject.targets + + + + + + + \ No newline at end of file diff --git a/Services/DataX.Metrics/DataX.Metrics.Ingestor/DataX.Metrics.Ingestor.csproj b/Services/DataX.Metrics/DataX.Metrics.Ingestor/DataX.Metrics.Ingestor.csproj index 073b6466b..466f86567 100644 --- a/Services/DataX.Metrics/DataX.Metrics.Ingestor/DataX.Metrics.Ingestor.csproj +++ b/Services/DataX.Metrics/DataX.Metrics.Ingestor/DataX.Metrics.Ingestor.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 True True win7-x64 @@ -20,14 +20,13 @@ - - + all runtime; build; native; contentfiles; analyzers - + @@ -49,14 +48,14 @@ - Microsoft + Microsoft400 StrongName - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Metrics/DataX.Metrics.Ingestor/Helper/SecretStore.cs b/Services/DataX.Metrics/DataX.Metrics.Ingestor/Helper/SecretStore.cs index 6e963050d..4bbdc0a18 100644 --- a/Services/DataX.Metrics/DataX.Metrics.Ingestor/Helper/SecretStore.cs +++ b/Services/DataX.Metrics/DataX.Metrics.Ingestor/Helper/SecretStore.cs @@ -1,7 +1,8 @@ -// ********************************************************************* +// ********************************************************************* // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using DataX.Contract; using DataX.ServiceHost.ServiceFabric; using DataX.Utilities.KeyVault; using System; @@ -20,28 +21,30 @@ internal class SecretsStore private SecretsStore() { - _keyVaultName = (string)ServiceFabricUtil.GetServiceKeyVaultName().Result; + _keyVaultName = ServiceFabricUtil.GetServiceKeyVaultName().Result?.ToString(); } public static SecretsStore Instance => _LazyInstance.Value; public async Task GetMetricsEventHubListenerConnectionStringAsync() { - return await GetSecretAsync(Utility.GetConfigValue("EventhubNamespaceConnectionstring")); + return await GetSecretAsync(ServiceFabricUtil.GetServiceFabricConfigSetting("EventhubNamespaceConnectionstring")); } public async Task GetMetricsStorageConnectionStringAsync() { - return await GetSecretAsync(Utility.GetConfigValue("StorageAccountConnectionstring")); + return await GetSecretAsync(ServiceFabricUtil.GetServiceFabricConfigSetting("StorageAccountConnectionstring")); } public async Task GetMetricsRedisConnectionStringAsync() { - return await GetSecretAsync(Utility.GetConfigValue("RedisCacheConnectionstring")); + return await GetSecretAsync(ServiceFabricUtil.GetServiceFabricConfigSetting("RedisCacheConnectionstring")); } - private async Task GetSecretAsync(string key) + private async Task GetSecretAsync(ApiResult setting) { + string key = setting.Result?.ToString(); + KeyVaultManager keyManager = new KeyVaultManager(); var secret = await keyManager.GetSecretStringAsync(_keyVaultName, key); return secret; diff --git a/Services/DataX.Metrics/DataX.Metrics.Ingestor/Helper/Utility.cs b/Services/DataX.Metrics/DataX.Metrics.Ingestor/Helper/Utility.cs index 787c826a2..ec18d1e1a 100644 --- a/Services/DataX.Metrics/DataX.Metrics.Ingestor/Helper/Utility.cs +++ b/Services/DataX.Metrics/DataX.Metrics.Ingestor/Helper/Utility.cs @@ -1,7 +1,8 @@ -// ********************************************************************* +// ********************************************************************* // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using System; using System.Fabric; namespace DataX.Metrics.Ingestor.Helper @@ -9,6 +10,11 @@ namespace DataX.Metrics.Ingestor.Helper /// /// Utility class /// + // TODO: Remove when configuration fully consolidated + [Obsolete("Use another method for configuration, such as: " + + nameof(ServiceHost.ServiceFabric.ServiceFabricUtil) + ", " + + nameof(Contract.Settings.DataXSettings) + ", or " + + nameof(Microsoft.Extensions.Configuration.IConfiguration))] public static class Utility { /// diff --git a/Services/DataX.Metrics/DataX.Metrics.Ingestor/Ingestor.cs b/Services/DataX.Metrics/DataX.Metrics.Ingestor/Ingestor.cs index 00d75cf7c..88147f972 100644 --- a/Services/DataX.Metrics/DataX.Metrics.Ingestor/Ingestor.cs +++ b/Services/DataX.Metrics/DataX.Metrics.Ingestor/Ingestor.cs @@ -133,8 +133,8 @@ private async Task StopListenerAsync(EventProcessorHost eventProcessorHost, Canc private async Task InitalizeEventProcessorHostAsync() { // start listening to the event hub - var eventHubName = Utility.GetConfigValue("EventHubName"); - var consumerGroup = Utility.GetConfigValue("ConsumerGroupName"); + var eventHubName = ServiceFabricUtil.GetServiceFabricConfigSetting("EventHubName").Result?.ToString(); + var consumerGroup = ServiceFabricUtil.GetServiceFabricConfigSetting("ConsumerGroupName").Result?.ToString(); var eventProcessorHost = new EventProcessorHost( eventHubName, diff --git a/Services/DataX.Metrics/DataX.Metrics.Ingestor/Startup.cs b/Services/DataX.Metrics/DataX.Metrics.Ingestor/Startup.cs index abf121f90..e4ebd4f3a 100644 --- a/Services/DataX.Metrics/DataX.Metrics.Ingestor/Startup.cs +++ b/Services/DataX.Metrics/DataX.Metrics.Ingestor/Startup.cs @@ -29,7 +29,7 @@ public Startup(IConfiguration configuration) public IConfiguration Configuration { get; } // This method gets called by the runtime. Use this method to add services to the container. - public static void ConfigureServices(IServiceCollection services) + public void ConfigureServices(IServiceCollection services) { services.Configure(options => { @@ -37,7 +37,7 @@ public static void ConfigureServices(IServiceCollection services) options.CheckConsentNeeded = context => true; options.MinimumSameSitePolicy = SameSiteMode.None; }); - StartUpUtil.ConfigureServices(services); + StartUpUtil.ConfigureServices(services, Configuration); services.AddMvc().SetCompatibilityVersion(CompatibilityVersion.Version_2_1); } diff --git a/Services/DataX.Metrics/DataX.Metrics.Tests/DataX.Metrics.Tests.csproj b/Services/DataX.Metrics/DataX.Metrics.Tests/DataX.Metrics.Tests.csproj index 804bb0ad1..bf85649f5 100644 --- a/Services/DataX.Metrics/DataX.Metrics.Tests/DataX.Metrics.Tests.csproj +++ b/Services/DataX.Metrics/DataX.Metrics.Tests/DataX.Metrics.Tests.csproj @@ -1,7 +1,7 @@ - netcoreapp2.1 + netcoreapp2.2 false @@ -21,7 +21,7 @@ all runtime; build; native; contentfiles; analyzers - + all runtime; build; native; contentfiles; analyzers @@ -37,7 +37,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Metrics/DataX.Metrics.sln b/Services/DataX.Metrics/DataX.Metrics.sln index 00176dc62..a85eb4600 100644 --- a/Services/DataX.Metrics/DataX.Metrics.sln +++ b/Services/DataX.Metrics/DataX.Metrics.sln @@ -1,101 +1,111 @@ - -Microsoft Visual Studio Solution File, Format Version 12.00 -# Visual Studio 15 -VisualStudioVersion = 15.0.28307.136 -MinimumVisualStudioVersion = 10.0.40219.1 -Project("{A07B5EB6-E848-4116-A8D0-A826331D98C6}") = "DataX.Metrics", "DataX.Metrics\DataX.Metrics.sfproj", "{0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Metrics.Ingestor", "DataX.Metrics.Ingestor\DataX.Metrics.Ingestor.csproj", "{5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.KeyVault", "..\DataX.Utilities\DataX.Utilities.KeyVault\DataX.Utilities.KeyVault.csproj", "{DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.Telemetry", "..\DataX.Utilities\DataX.Utilities.Telemetry\DataX.Utilities.Telemetry.csproj", "{2A1EBF8C-9071-415E-8AE4-12C2043C4F28}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Metrics.Tests", "DataX.Metrics.Tests\DataX.Metrics.Tests.csproj", "{9B18CDDA-A745-4A97-A83E-97E248850BC7}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Contract", "..\DataX.Contract\DataX.Contract.csproj", "{EA0BF604-431B-4EA0-90EF-31AC7683B6D0}" -EndProject -Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.ServiceHost.ServiceFabric", "..\DataX.ServiceHost\DataX.ServiceHost.ServiceFabric\DataX.ServiceHost.ServiceFabric.csproj", "{52FB95BC-098F-4010-B82A-AA8097C7D0FF}" -EndProject -Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "DataX.Utility.Nuget", "..\DataX.Utilities\DataX.Utility.Nuget\DataX.Utility.Nuget.csproj", "{91C04219-9332-44EB-BF85-B07FB7718AD4}" -EndProject -Global - GlobalSection(SolutionConfigurationPlatforms) = preSolution - Debug|Any CPU = Debug|Any CPU - Debug|x64 = Debug|x64 - Release|Any CPU = Release|Any CPU - Release|x64 = Release|x64 - EndGlobalSection - GlobalSection(ProjectConfigurationPlatforms) = postSolution - {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Debug|Any CPU.ActiveCfg = Debug|x64 - {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Debug|x64.ActiveCfg = Debug|x64 - {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Debug|x64.Build.0 = Debug|x64 - {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Debug|x64.Deploy.0 = Debug|x64 - {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Release|Any CPU.ActiveCfg = Release|x64 - {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Release|x64.ActiveCfg = Release|x64 - {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Release|x64.Build.0 = Release|x64 - {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Release|x64.Deploy.0 = Release|x64 - {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Debug|Any CPU.Build.0 = Debug|Any CPU - {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Debug|x64.ActiveCfg = Debug|Any CPU - {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Debug|x64.Build.0 = Debug|Any CPU - {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Release|Any CPU.ActiveCfg = Release|Any CPU - {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Release|Any CPU.Build.0 = Release|Any CPU - {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Release|x64.ActiveCfg = Release|Any CPU - {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Release|x64.Build.0 = Release|Any CPU - {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Debug|Any CPU.Build.0 = Debug|Any CPU - {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Debug|x64.ActiveCfg = Debug|Any CPU - {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Debug|x64.Build.0 = Debug|Any CPU - {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Release|Any CPU.ActiveCfg = Release|Any CPU - {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Release|Any CPU.Build.0 = Release|Any CPU - {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Release|x64.ActiveCfg = Release|Any CPU - {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Release|x64.Build.0 = Release|Any CPU - {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Debug|Any CPU.Build.0 = Debug|Any CPU - {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Debug|x64.ActiveCfg = Debug|Any CPU - {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Debug|x64.Build.0 = Debug|Any CPU - {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Release|Any CPU.ActiveCfg = Release|Any CPU - {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Release|Any CPU.Build.0 = Release|Any CPU - {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Release|x64.ActiveCfg = Release|Any CPU - {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Release|x64.Build.0 = Release|Any CPU - {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Debug|Any CPU.Build.0 = Debug|Any CPU - {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Debug|x64.ActiveCfg = Debug|Any CPU - {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Debug|x64.Build.0 = Debug|Any CPU - {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Release|Any CPU.ActiveCfg = Release|Any CPU - {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Release|Any CPU.Build.0 = Release|Any CPU - {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Release|x64.ActiveCfg = Release|Any CPU - {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Release|x64.Build.0 = Release|Any CPU - {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Debug|Any CPU.Build.0 = Debug|Any CPU - {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Debug|x64.ActiveCfg = Debug|Any CPU - {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Debug|x64.Build.0 = Debug|Any CPU - {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Release|Any CPU.ActiveCfg = Release|Any CPU - {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Release|Any CPU.Build.0 = Release|Any CPU - {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Release|x64.ActiveCfg = Release|Any CPU - {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Release|x64.Build.0 = Release|Any CPU - {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Debug|Any CPU.Build.0 = Debug|Any CPU - {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Debug|x64.ActiveCfg = Debug|Any CPU - {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Debug|x64.Build.0 = Debug|Any CPU - {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Release|Any CPU.ActiveCfg = Release|Any CPU - {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Release|Any CPU.Build.0 = Release|Any CPU - {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Release|x64.ActiveCfg = Release|Any CPU - {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Release|x64.Build.0 = Release|Any CPU - {91C04219-9332-44EB-BF85-B07FB7718AD4}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {91C04219-9332-44EB-BF85-B07FB7718AD4}.Debug|Any CPU.Build.0 = Debug|Any CPU - {91C04219-9332-44EB-BF85-B07FB7718AD4}.Debug|x64.ActiveCfg = Debug|Any CPU - {91C04219-9332-44EB-BF85-B07FB7718AD4}.Debug|x64.Build.0 = Debug|Any CPU - {91C04219-9332-44EB-BF85-B07FB7718AD4}.Release|Any CPU.ActiveCfg = Release|Any CPU - {91C04219-9332-44EB-BF85-B07FB7718AD4}.Release|Any CPU.Build.0 = Release|Any CPU - {91C04219-9332-44EB-BF85-B07FB7718AD4}.Release|x64.ActiveCfg = Release|Any CPU - {91C04219-9332-44EB-BF85-B07FB7718AD4}.Release|x64.Build.0 = Release|Any CPU - EndGlobalSection - GlobalSection(SolutionProperties) = preSolution - HideSolutionNode = FALSE - EndGlobalSection - GlobalSection(ExtensibilityGlobals) = postSolution - SolutionGuid = {CD1D74C0-B681-419E-83F9-8FFE00EFA6A6} - EndGlobalSection -EndGlobal + +Microsoft Visual Studio Solution File, Format Version 12.00 +# Visual Studio 15 +VisualStudioVersion = 15.0.28307.136 +MinimumVisualStudioVersion = 10.0.40219.1 +Project("{A07B5EB6-E848-4116-A8D0-A826331D98C6}") = "DataX.Metrics", "DataX.Metrics\DataX.Metrics.sfproj", "{0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Metrics.Ingestor", "DataX.Metrics.Ingestor\DataX.Metrics.Ingestor.csproj", "{5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.KeyVault", "..\DataX.Utilities\DataX.Utilities.KeyVault\DataX.Utilities.KeyVault.csproj", "{DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.Telemetry", "..\DataX.Utilities\DataX.Utilities.Telemetry\DataX.Utilities.Telemetry.csproj", "{2A1EBF8C-9071-415E-8AE4-12C2043C4F28}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Metrics.Tests", "DataX.Metrics.Tests\DataX.Metrics.Tests.csproj", "{9B18CDDA-A745-4A97-A83E-97E248850BC7}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Contract", "..\DataX.Contract\DataX.Contract.csproj", "{EA0BF604-431B-4EA0-90EF-31AC7683B6D0}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.ServiceHost.ServiceFabric", "..\DataX.ServiceHost\DataX.ServiceHost.ServiceFabric\DataX.ServiceHost.ServiceFabric.csproj", "{52FB95BC-098F-4010-B82A-AA8097C7D0FF}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.Nuget", "..\DataX.Utilities\DataX.Utility.Nuget\DataX.Utility.Nuget.csproj", "{91C04219-9332-44EB-BF85-B07FB7718AD4}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.ServiceHost", "..\DataX.ServiceHost\DataX.ServiceHost\DataX.ServiceHost.csproj", "{AB81CD31-F031-496F-B810-7FBED8A906CC}" +EndProject +Global + GlobalSection(SolutionConfigurationPlatforms) = preSolution + Debug|Any CPU = Debug|Any CPU + Debug|x64 = Debug|x64 + Release|Any CPU = Release|Any CPU + Release|x64 = Release|x64 + EndGlobalSection + GlobalSection(ProjectConfigurationPlatforms) = postSolution + {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Debug|Any CPU.ActiveCfg = Debug|x64 + {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Debug|x64.ActiveCfg = Debug|x64 + {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Debug|x64.Build.0 = Debug|x64 + {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Debug|x64.Deploy.0 = Debug|x64 + {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Release|Any CPU.ActiveCfg = Release|x64 + {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Release|x64.ActiveCfg = Release|x64 + {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Release|x64.Build.0 = Release|x64 + {0DE74D1C-16D6-42DD-BEAF-DDF8D556DBFE}.Release|x64.Deploy.0 = Release|x64 + {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Debug|Any CPU.Build.0 = Debug|Any CPU + {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Debug|x64.ActiveCfg = Debug|Any CPU + {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Debug|x64.Build.0 = Debug|Any CPU + {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Release|Any CPU.ActiveCfg = Release|Any CPU + {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Release|Any CPU.Build.0 = Release|Any CPU + {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Release|x64.ActiveCfg = Release|Any CPU + {5AB3277B-5BB8-4527-82F9-9B9D78E12A3E}.Release|x64.Build.0 = Release|Any CPU + {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Debug|Any CPU.Build.0 = Debug|Any CPU + {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Debug|x64.ActiveCfg = Debug|Any CPU + {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Debug|x64.Build.0 = Debug|Any CPU + {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Release|Any CPU.ActiveCfg = Release|Any CPU + {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Release|Any CPU.Build.0 = Release|Any CPU + {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Release|x64.ActiveCfg = Release|Any CPU + {DB5B770D-3F2D-4B99-B7F0-8462B6B1A11A}.Release|x64.Build.0 = Release|Any CPU + {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Debug|Any CPU.Build.0 = Debug|Any CPU + {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Debug|x64.ActiveCfg = Debug|Any CPU + {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Debug|x64.Build.0 = Debug|Any CPU + {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Release|Any CPU.ActiveCfg = Release|Any CPU + {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Release|Any CPU.Build.0 = Release|Any CPU + {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Release|x64.ActiveCfg = Release|Any CPU + {2A1EBF8C-9071-415E-8AE4-12C2043C4F28}.Release|x64.Build.0 = Release|Any CPU + {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Debug|Any CPU.Build.0 = Debug|Any CPU + {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Debug|x64.ActiveCfg = Debug|Any CPU + {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Debug|x64.Build.0 = Debug|Any CPU + {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Release|Any CPU.ActiveCfg = Release|Any CPU + {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Release|Any CPU.Build.0 = Release|Any CPU + {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Release|x64.ActiveCfg = Release|Any CPU + {9B18CDDA-A745-4A97-A83E-97E248850BC7}.Release|x64.Build.0 = Release|Any CPU + {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Debug|Any CPU.Build.0 = Debug|Any CPU + {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Debug|x64.ActiveCfg = Debug|Any CPU + {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Debug|x64.Build.0 = Debug|Any CPU + {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Release|Any CPU.ActiveCfg = Release|Any CPU + {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Release|Any CPU.Build.0 = Release|Any CPU + {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Release|x64.ActiveCfg = Release|Any CPU + {EA0BF604-431B-4EA0-90EF-31AC7683B6D0}.Release|x64.Build.0 = Release|Any CPU + {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Debug|Any CPU.Build.0 = Debug|Any CPU + {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Debug|x64.ActiveCfg = Debug|Any CPU + {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Debug|x64.Build.0 = Debug|Any CPU + {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Release|Any CPU.ActiveCfg = Release|Any CPU + {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Release|Any CPU.Build.0 = Release|Any CPU + {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Release|x64.ActiveCfg = Release|Any CPU + {52FB95BC-098F-4010-B82A-AA8097C7D0FF}.Release|x64.Build.0 = Release|Any CPU + {91C04219-9332-44EB-BF85-B07FB7718AD4}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {91C04219-9332-44EB-BF85-B07FB7718AD4}.Debug|Any CPU.Build.0 = Debug|Any CPU + {91C04219-9332-44EB-BF85-B07FB7718AD4}.Debug|x64.ActiveCfg = Debug|Any CPU + {91C04219-9332-44EB-BF85-B07FB7718AD4}.Debug|x64.Build.0 = Debug|Any CPU + {91C04219-9332-44EB-BF85-B07FB7718AD4}.Release|Any CPU.ActiveCfg = Release|Any CPU + {91C04219-9332-44EB-BF85-B07FB7718AD4}.Release|Any CPU.Build.0 = Release|Any CPU + {91C04219-9332-44EB-BF85-B07FB7718AD4}.Release|x64.ActiveCfg = Release|Any CPU + {91C04219-9332-44EB-BF85-B07FB7718AD4}.Release|x64.Build.0 = Release|Any CPU + {AB81CD31-F031-496F-B810-7FBED8A906CC}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {AB81CD31-F031-496F-B810-7FBED8A906CC}.Debug|Any CPU.Build.0 = Debug|Any CPU + {AB81CD31-F031-496F-B810-7FBED8A906CC}.Debug|x64.ActiveCfg = Debug|Any CPU + {AB81CD31-F031-496F-B810-7FBED8A906CC}.Debug|x64.Build.0 = Debug|Any CPU + {AB81CD31-F031-496F-B810-7FBED8A906CC}.Release|Any CPU.ActiveCfg = Release|Any CPU + {AB81CD31-F031-496F-B810-7FBED8A906CC}.Release|Any CPU.Build.0 = Release|Any CPU + {AB81CD31-F031-496F-B810-7FBED8A906CC}.Release|x64.ActiveCfg = Release|Any CPU + {AB81CD31-F031-496F-B810-7FBED8A906CC}.Release|x64.Build.0 = Release|Any CPU + EndGlobalSection + GlobalSection(SolutionProperties) = preSolution + HideSolutionNode = FALSE + EndGlobalSection + GlobalSection(ExtensibilityGlobals) = postSolution + SolutionGuid = {CD1D74C0-B681-419E-83F9-8FFE00EFA6A6} + EndGlobalSection +EndGlobal diff --git a/Services/DataX.Metrics/DataXMetrics.nuspec b/Services/DataX.Metrics/DataXMetrics.nuspec index 3bf95a812..b2865aab3 100644 --- a/Services/DataX.Metrics/DataXMetrics.nuspec +++ b/Services/DataX.Metrics/DataXMetrics.nuspec @@ -16,7 +16,7 @@ - + \ No newline at end of file diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/DataXAuthConstants.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/DataXAuthConstants.cs new file mode 100644 index 000000000..f7fc06f86 --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/DataXAuthConstants.cs @@ -0,0 +1,23 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Utilities.Web; +using System; +using System.Collections.Generic; +using System.Text; + +namespace DataX.ServiceHost.AspNetCore.Authorization +{ + /// + /// Constants used in DataX auth scenarios + /// + public static class DataXAuthConstants + { + public const string PolicyPrefix = "DataXAuth_"; + + public static string WriterPolicyName { get; } = PolicyPrefix + RolesCheck.WriterRoleName; + + public static string ReaderPolicyName { get; } = PolicyPrefix + RolesCheck.ReaderRoleName; + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/DataXAuthorizeAttribute.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/DataXAuthorizeAttribute.cs new file mode 100644 index 000000000..d9b601dff --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/DataXAuthorizeAttribute.cs @@ -0,0 +1,22 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Utilities.Web; +using Microsoft.AspNetCore.Authorization; +using System; +using System.Collections.Generic; +using System.Text; +using System.Threading.Tasks; + +namespace DataX.ServiceHost.AspNetCore.Authorization +{ + /// + public abstract class DataXAuthorizeAttribute : AuthorizeAttribute + { + public DataXAuthorizeAttribute() + { + Policy = DataXAuthConstants.PolicyPrefix; + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/DataXPolicyBuilder.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/DataXPolicyBuilder.cs new file mode 100644 index 000000000..3178a564d --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/DataXPolicyBuilder.cs @@ -0,0 +1,87 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +namespace DataX.ServiceHost.AspNetCore.Authorization +{ + using DataX.ServiceHost.Authorization; + using DataX.Contract.Settings; + using Microsoft.AspNetCore.Authorization; + using System; + using System.Linq; + + /// + /// This class is meant to simplify the syntax for adding requirements for policies + /// + internal class DataXPolicyBuilder + { + private readonly AuthorizationOptions _options; + private readonly DataXSettings _settings; + private readonly Action _configurePolicy; + + public DataXPolicyBuilder( + AuthorizationOptions options, + DataXSettings settings, + Action configurePolicy) + { + _options = options; + _settings = settings; + _configurePolicy = configurePolicy; + } + + public DataXPolicyBuilder AddPolicy(string name) + where TRequirement : DataXAuthRequirement, new() + { + _options.AddPolicy(name, DataXPolicy); + + return this; + } + + private void DataXPolicy(AuthorizationPolicyBuilder policy) + where TDataXRequirement : DataXAuthRequirement, new() + { + AddDataXRequirements(policy); + } + + /// + /// Adds the basic DataX auth policy to the builder + /// + private AuthorizationPolicyBuilder AddDataXRequirements(AuthorizationPolicyBuilder policy) + where TDataXRequirement : DataXAuthRequirement, new() + { + // We don't want to add the same requirement in again. + // If it does exist and the settings changed, then we want to make sure the new settings are used + RemoveDataXRequirements(policy); + + var requirement = new TDataXRequirement() + { + Settings = _settings + }; + + if (!_settings.EnableOneBox) + { + policy.RequireAuthenticatedUser(); + } + + policy.AddRequirements(requirement); + _configurePolicy?.Invoke(policy); + + return policy; + } + + /// + /// Removes the DataXRequirements set in the policy builder if they exist. + /// + private static AuthorizationPolicyBuilder RemoveDataXRequirements(AuthorizationPolicyBuilder policy) + { + var requirements = policy.Requirements.Where(req => req is DataXAuthRequirement); + + foreach (var req in requirements) + { + policy.Requirements.Remove(req); + } + + return policy; + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Extensions/DataXAuthorizationExtensions.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Extensions/DataXAuthorizationExtensions.cs new file mode 100644 index 000000000..34770e9cf --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Extensions/DataXAuthorizationExtensions.cs @@ -0,0 +1,55 @@ +using DataX.Contract.Settings; +using DataX.Utilities.Web; +using Microsoft.AspNetCore.Authorization; +using Microsoft.Extensions.DependencyInjection; +using System; +using System.Collections.Generic; +using System.Text; +using System.Linq; +using DataX.ServiceHost.AspNetCore.Authorization.Requirements; + +namespace DataX.ServiceHost.AspNetCore.Authorization.Extensions +{ + public static class DataXAuthorizationExtensions + { + /// + /// Adds DataX default authentication to the given service collection. + /// + /// The service collection to add to + /// The modified service collection + public static IServiceCollection AddDataXAuthorization(this IServiceCollection services) + { + return services.AddDataXAuthorization(null); + } + + /// + /// Adds DataX default authentication to the given service collection. + /// + /// The service collection to add to + /// An action to configure supplement policy configuration + /// The modified service collection + public static IServiceCollection AddDataXAuthorization(this IServiceCollection services, Action configurePolicy) + { + var settings = services.BuildServiceProvider().GetService(); + + // EnableOneBox scenario as it requires the least configuration and we can't assume cloud connection settings + if (settings == null) + { + settings = new DataXSettings() + { + EnableOneBox = true, + LocalRoot = "", + MetricsHttpEndpoint = "http://localhost:2020/", + SparkHome = "", + }; + } + + return services.AddAuthorization(options => + { + new DataXPolicyBuilder(options, settings, configurePolicy) + .AddPolicy(DataXAuthConstants.WriterPolicyName) + .AddPolicy(DataXAuthConstants.ReaderPolicyName); + }); + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Requirements/DataXReaderRequirement.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Requirements/DataXReaderRequirement.cs new file mode 100644 index 000000000..7c50f00b2 --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Requirements/DataXReaderRequirement.cs @@ -0,0 +1,25 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +namespace DataX.ServiceHost.AspNetCore.Authorization.Requirements +{ + using DataX.Contract.Settings; + using DataX.Utilities.Web; + using Microsoft.AspNetCore.Authorization; + + /// + internal class DataXReaderRequirement : DataXWriterRequirement + { + public DataXReaderRequirement() { } + + public DataXReaderRequirement(DataXSettings settings) + : base(settings) { } + + /// + protected override bool IsAuthorized(AuthorizationHandlerContext context, DataXSettings settings) + { + return base.IsAuthorized(context, settings) || context.User.IsInRole(RolesCheck.ReaderRoleName); + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Requirements/DataXWriterRequirement.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Requirements/DataXWriterRequirement.cs new file mode 100644 index 000000000..6f47bc55d --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Requirements/DataXWriterRequirement.cs @@ -0,0 +1,22 @@ +namespace DataX.ServiceHost.AspNetCore.Authorization.Requirements +{ + using DataX.ServiceHost.Authorization.Requirements; + using DataX.Contract.Settings; + using DataX.Utilities.Web; + using Microsoft.AspNetCore.Authorization; + + /// + internal class DataXWriterRequirement : DataXOneBoxRequirement + { + public DataXWriterRequirement() { } + + public DataXWriterRequirement(DataXSettings settings) + : base(settings) { } + + /// + protected override bool IsAuthorized(AuthorizationHandlerContext context, DataXSettings settings) + { + return base.IsAuthorized(context, settings) || context.User.IsInRole(RolesCheck.WriterRoleName); + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Roles/DataXReaderAttribute.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Roles/DataXReaderAttribute.cs new file mode 100644 index 000000000..f952bf5cc --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Roles/DataXReaderAttribute.cs @@ -0,0 +1,19 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using System; +using System.Collections.Generic; +using System.Text; + +namespace DataX.ServiceHost.AspNetCore.Authorization.Roles +{ + /// + public class DataXReaderAttribute : DataXAuthorizeAttribute + { + public DataXReaderAttribute() + { + Policy = DataXAuthConstants.ReaderPolicyName; + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Roles/DataXWriterAttribute.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Roles/DataXWriterAttribute.cs new file mode 100644 index 000000000..53a28b424 --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Authorization/Roles/DataXWriterAttribute.cs @@ -0,0 +1,19 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using System; +using System.Collections.Generic; +using System.Text; + +namespace DataX.ServiceHost.AspNetCore.Authorization.Roles +{ + /// + public class DataXWriterAttribute : DataXAuthorizeAttribute + { + public DataXWriterAttribute() + { + Policy = DataXAuthConstants.WriterPolicyName; + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/DataX.ServiceHost.AspNetCore.csproj b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/DataX.ServiceHost.AspNetCore.csproj index 5ebf2fe80..548a76323 100644 --- a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/DataX.ServiceHost.AspNetCore.csproj +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/DataX.ServiceHost.AspNetCore.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -17,8 +17,9 @@ + - + all runtime; build; native; contentfiles; analyzers @@ -31,7 +32,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Extensions/DataXApplicationBuilderExtensions.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Extensions/DataXApplicationBuilderExtensions.cs new file mode 100644 index 000000000..da7698843 --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Extensions/DataXApplicationBuilderExtensions.cs @@ -0,0 +1,35 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +namespace DataX.ServiceHost.AspNetCore.Extensions +{ + using Microsoft.AspNetCore.Builder; + using Microsoft.AspNetCore.Hosting; + using Microsoft.Extensions.DependencyInjection; + + /// + /// Extensions for IApplicationBuilder + /// + public static class DataXApplicationBuilderExtensions + { + /// + /// Configures and ApplicationBuilder with DataX defaults. + /// + /// The ApplicationBuilder to be configured + /// The configured ApplicationBuilder + public static IApplicationBuilder UseDataXApplicationDefaults(this IApplicationBuilder app) + { + var env = app.ApplicationServices.GetRequiredService(); + + if (env.IsDevelopment()) + { + app.UseDeveloperExceptionPage(); + } + + app.UseMvc(); + + return app; + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Extensions/DataXServiceCollectionExtensions.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Extensions/DataXServiceCollectionExtensions.cs new file mode 100644 index 000000000..5f5f998a2 --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Extensions/DataXServiceCollectionExtensions.cs @@ -0,0 +1,66 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.ServiceHost.AspNetCore.Startup; +using Microsoft.AspNetCore.Hosting; +using Microsoft.AspNetCore.Mvc.ApplicationParts; +using Microsoft.Extensions.DependencyInjection; +using System; +using System.Linq; +using System.Collections.Generic; +using System.Text; + +namespace DataX.ServiceHost.AspNetCore.Extensions +{ + /// + /// Extensions for IServiceCollection + /// + public static class DataXServiceCollectionExtensions + { + /// + /// Adds the specified DataX startup filter to the provides services. + /// + /// A type deriving from + /// The services to add the DataX startup to + /// Optional startup instance + /// The configured services + public static IServiceCollection AddDataXStartup(this IServiceCollection services, TStartup startup = null) + where TStartup : DataXServiceStartup, new() + { + if (services == null) + { + throw new ArgumentNullException(nameof(services)); + } + + startup = startup ?? new TStartup(); + + startup.ConfigureServices(services); + + services.AddTransient(_ => startup); + + return services; + } + + /// + /// Removes DataX controllers based on a provided DataX startup type + /// + /// Concrete startup type for assembly reference + /// The application part manager to remove DataX controllers from + /// The modified application part manager + public static ApplicationPartManager RemoveDataXController(this ApplicationPartManager manager) + where TStartup : DataXServiceStartup, new() + { + var startupAssembly = typeof(TStartup).Assembly; + + var partsToRemove = manager.ApplicationParts.OfType().Where(p => p.Assembly == startupAssembly); + + foreach (var part in partsToRemove) + { + manager.ApplicationParts.Remove(part); + } + + return manager; + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Extensions/DataXWebHostBuilderExtensions.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Extensions/DataXWebHostBuilderExtensions.cs new file mode 100644 index 000000000..37e6cfa75 --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Extensions/DataXWebHostBuilderExtensions.cs @@ -0,0 +1,127 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +namespace DataX.ServiceHost.AspNetCore.Extensions +{ + using DataX.Contract.Settings; + using DataX.ServiceHost.AspNetCore.Startup; + using DataX.ServiceHost.ServiceFabric.Extensions.Configuration; + using DataX.Utilities.Telemetry; + using Microsoft.AspNetCore.Authentication.JwtBearer; + using Microsoft.AspNetCore.Hosting; + using Microsoft.Extensions.Configuration; + using Microsoft.Extensions.DependencyInjection; + using System.IO; + + /// + /// Extensions for IWebHostBuilder + /// + public static class DataXWebHostBuilderExtensions + { + /// + /// Configures the WebHostBuilder with DataX defaults. + /// + /// The host builder to be configured + /// The configured host builder + public static IWebHostBuilder UseDataXDefaultConfiguration(this IWebHostBuilder hostBuilder) + { + return hostBuilder + .UseKestrel() + .UseContentRoot(Directory.GetCurrentDirectory()) + .UseDataXDefaultAppConfiguration() + .UseDataXDefaultConfigureServices(); + } + + /// + /// Configures the WebHostBuilder with DataX defaults and a DataX startup filter. + /// + /// DataX concrete startup filter type inheriting from + /// The host builder to be configured + /// Optional startup instance to be used. + /// The configured host builder + public static IWebHostBuilder UseDataXDefaultConfiguration(this IWebHostBuilder hostBuilder, TStartup startup = null) + where TStartup : DataXServiceStartup, new() + { + return hostBuilder + .UseDataXDefaultConfiguration() + .UseDataXStartup(startup); + } + + /// + /// Configures the WebHostBuilder with the DataX default IConfigurationBuilder settings. + /// + /// The host builder to be configured + /// The configured host builder + public static IWebHostBuilder UseDataXDefaultAppConfiguration(this IWebHostBuilder hostBuilder) + { + void ConfigureAppConfiguration(WebHostBuilderContext context, IConfigurationBuilder builder) + { + var env = context.HostingEnvironment; + + builder = builder + .SetBasePath(env.ContentRootPath) + .AddJsonFile("appsettings.json", optional: false, reloadOnChange: true) + .AddJsonFile($"appsettings.{env.EnvironmentName}.json", optional: true) + .AddServiceFabricSettings("Config", DataXSettingsConstants.DataX) + .AddEnvironmentVariables(); + } + + return hostBuilder.ConfigureAppConfiguration(ConfigureAppConfiguration); + } + + /// + /// Configures the WebHostBuilder with the DataX default services. + /// + /// The host builder to be configured + /// The configured host builder + public static IWebHostBuilder UseDataXDefaultConfigureServices(this IWebHostBuilder hostBuilder) + { + void ConfigureServices(IServiceCollection services) + { + var config = services.BuildServiceProvider().GetRequiredService(); + + // Add DataX settings to be picked up automatically + var settings = config.GetSection(DataXSettingsConstants.ServiceEnvironment).Get(); + + services.AddSingleton(settings); + + // Configures AppInsights logging + StartUpUtil.ConfigureServices(services, config); + + // Adds JWT Auth + var bearerOptions = new JwtBearerOptions(); + + config.GetSection("JwtBearerOptions").Bind(bearerOptions); + + services + .AddAuthentication(options => + { + options.DefaultAuthenticateScheme = JwtBearerDefaults.AuthenticationScheme; + options.DefaultChallengeScheme = JwtBearerDefaults.AuthenticationScheme; + options.DefaultScheme = JwtBearerDefaults.AuthenticationScheme; + }) + .AddJwtBearer(options => + { + options.Audience = bearerOptions.Audience; + options.Authority = bearerOptions.Authority; + }); + } + + return hostBuilder.ConfigureServices(ConfigureServices); + } + + /// + /// Adds a DataX startup filter to a WebHostBuilder. + /// + /// DataX concrete startup filter type inheriting from + /// The host builder to be configured + /// Optional startup instance to be used. + /// The configured host builder + public static IWebHostBuilder UseDataXStartup(this IWebHostBuilder hostBuilder, TStartup startup = null) + where TStartup : DataXServiceStartup, new() + { + return hostBuilder.ConfigureServices(services => services.AddDataXStartup(startup)); + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/StartUpBase.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/StartUpBase.cs index 68db4e906..f97750b84 100644 --- a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/StartUpBase.cs +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/StartUpBase.cs @@ -2,6 +2,8 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* +using DataX.ServiceHost.ServiceFabric.Extensions.Configuration; +using DataX.Contract.Settings; using DataX.Utilities.Telemetry; using Microsoft.AspNetCore.Builder; using Microsoft.AspNetCore.Hosting; @@ -20,7 +22,8 @@ public StartUpBase(IHostingEnvironment env) .SetBasePath(env.ContentRootPath) .AddJsonFile("appsettings.json", optional: false, reloadOnChange: true) .AddJsonFile($"appsettings.{env.EnvironmentName}.json", optional: true) - .AddEnvironmentVariables(); + .AddServiceFabricSettings("Config", DataXSettingsConstants.DataX) + .AddEnvironmentVariables(); Configuration = builder.Build(); } @@ -30,7 +33,7 @@ public StartUpBase(IHostingEnvironment env) public void ConfigureServices(IServiceCollection services) { services.AddMvc().SetCompatibilityVersion(CompatibilityVersion.Version_2_1); - StartUpUtil.ConfigureServices(services); + StartUpUtil.ConfigureServices(services, Configuration); } // This method gets called by the runtime. Use this method to configure the HTTP request pipeline. diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Startup/DataXServiceStartup.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Startup/DataXServiceStartup.cs new file mode 100644 index 000000000..f079bd776 --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Startup/DataXServiceStartup.cs @@ -0,0 +1,95 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using System; +using System.Linq; +using System.Collections.Generic; +using System.Text; +using DataX.Contract.Settings; +using DataX.ServiceHost.AspNetCore.Authorization.Extensions; +using DataX.ServiceHost.ServiceFabric.Authorization; +using Microsoft.AspNetCore.Builder; +using Microsoft.AspNetCore.Hosting; +using Microsoft.Extensions.Configuration; +using Microsoft.Extensions.DependencyInjection; +using Microsoft.Extensions.DependencyInjection.Extensions; +using Microsoft.Extensions.Logging; +using Microsoft.AspNetCore.Mvc.ApplicationParts; + +namespace DataX.ServiceHost.AspNetCore.Startup +{ + /// + /// A basic abstract implementation of DataX startup to take care of common configuration. + /// + public abstract class DataXServiceStartup : IDataXServiceStartup + { + protected DataXSettings Settings { get; private set; } + + public DataXServiceStartup() { } + + public DataXServiceStartup(DataXSettings settings) + { + Settings = settings; + } + + /// + public virtual void ConfigureServices(IServiceCollection services) + { + if (Settings == null) + { + var provider = services.BuildServiceProvider(); + Settings = provider.GetService(); + + // Let's look for the default settings + if (Settings == null) + { + Settings = provider.GetService()?.GetSection(DataXSettingsConstants.ServiceEnvironment).Get(); + + if (Settings != null) + { + services.TryAddSingleton(Settings); + } + } + } + else + { + services.TryAddSingleton(Settings); + } + + services + .AddDataXAuthorization(DataXDefaultGatewayPolicy.ConfigurePolicy) + .AddMvc(); + } + + /// + public void Configure(IApplicationBuilder app) + { + var hostingEnvironment = app.ApplicationServices.GetService(); + var loggerFactory = app.ApplicationServices.GetService(); + + Configure(app, hostingEnvironment, loggerFactory); + } + + /// + protected virtual void Configure(IApplicationBuilder app, IHostingEnvironment env, ILoggerFactory loggerFactory) + { + app.Use(async (context, next) => + { + context.Response.Headers.Add("X-Content-Type-Options", new string[] { "nosniff" }); + await next(); + }); + app.UseAuthentication(); + app.UseMvc(); + } + + public Action Configure(Action next) + { + return app => + { + Configure(app); + next(app); + }; + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Startup/IDataXServiceStartup.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Startup/IDataXServiceStartup.cs new file mode 100644 index 000000000..95b19e1c8 --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.AspNetCore/Startup/IDataXServiceStartup.cs @@ -0,0 +1,29 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +namespace DataX.ServiceHost.AspNetCore.Startup +{ + using Microsoft.AspNetCore.Builder; + using Microsoft.AspNetCore.Hosting; + using Microsoft.Extensions.DependencyInjection; + + /// + /// Provides a common interface for DataX startup classes to use for instantiation in ASP.NET Core + /// + public interface IDataXServiceStartup : IStartupFilter + { + // These methods follow IStartup's interface, + // but it's not extended as we don't want autoload from default WebHost builder + + /// + /// This method gets called by the runtime. Use this method to add services to the container. + /// + void ConfigureServices(IServiceCollection services); + + /// + /// This method gets called by the runtime. Use this method to configure the HTTP request pipeline. + /// + void Configure(IApplicationBuilder app); + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.ServiceFabric/Authorization/DataXDefaultGatewayPolicy.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.ServiceFabric/Authorization/DataXDefaultGatewayPolicy.cs new file mode 100644 index 000000000..3ada18d59 --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.ServiceFabric/Authorization/DataXDefaultGatewayPolicy.cs @@ -0,0 +1,42 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +namespace DataX.ServiceHost.ServiceFabric.Authorization +{ + using DataX.ServiceHost.Authorization; + using DataX.Contract.Settings; + using Microsoft.AspNetCore.Authorization; + using Microsoft.AspNetCore.Authorization.Infrastructure; + using System.Linq; + + /// + public class DataXDefaultGatewayPolicy : DataXAuthRequirement + { + private DataXDefaultGatewayPolicy() { } + + /// + /// Configures the default, recommended policy for running in ServiceFabric behind Gateway + /// + /// The AuthorizationPolicyBuilder to modify + public static void ConfigurePolicy(AuthorizationPolicyBuilder builder) + { + // ServiceFabric setup is the only setup with the Gateway + if(HostUtil.InServiceFabric) + { + // Allow anonymous calls; RoleCheck will handle the checking in the controller + builder.Requirements = builder.Requirements.Where(r => !(r is DenyAnonymousAuthorizationRequirement || r is DataXAuthRequirement)).ToList(); + + // Ensure the OneBox check + builder.AddRequirements(new DataXDefaultGatewayPolicy()); + } + } + + /// + protected override bool IsAuthorized(AuthorizationHandlerContext context, DataXSettings settings) + { + // If OneBox, no auth. If in SF, no auth (handled in Gateway). True in both scenarios, so returning true. + return true; + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.ServiceFabric/DataX.ServiceHost.ServiceFabric.csproj b/Services/DataX.ServiceHost/DataX.ServiceHost.ServiceFabric/DataX.ServiceHost.ServiceFabric.csproj index d2ec7b362..d08a3f00b 100644 --- a/Services/DataX.ServiceHost/DataX.ServiceHost.ServiceFabric/DataX.ServiceHost.ServiceFabric.csproj +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.ServiceFabric/DataX.ServiceHost.ServiceFabric.csproj @@ -16,10 +16,12 @@ + all runtime; build; native; contentfiles; analyzers + all @@ -29,11 +31,12 @@ + - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.ServiceFabric/Extensions/Configuration/ServiceFabricConfigurationBuilderExtensions.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.ServiceFabric/Extensions/Configuration/ServiceFabricConfigurationBuilderExtensions.cs new file mode 100644 index 000000000..679e7eb8f --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.ServiceFabric/Extensions/Configuration/ServiceFabricConfigurationBuilderExtensions.cs @@ -0,0 +1,33 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +namespace DataX.ServiceHost.ServiceFabric.Extensions.Configuration +{ + using Microsoft.Extensions.Configuration; + using System; + + public static class ServiceFabricConfigurationBuilderExtensions + { + /// + /// Adds the specified service fabric settings to the configuration builder. If service fabric is not detected, nothing is added. + /// + /// The configuration package object name + /// An optional prefix to add to the configurations added to the builder. e.g. "MySettings:" + public static IConfigurationBuilder AddServiceFabricSettings(this IConfigurationBuilder configurationBuilder, string packageName, string configPrefix = "") + { + if (configurationBuilder == null) + { + throw new ArgumentNullException(nameof(configurationBuilder)); + } + + if (HostUtil.InServiceFabric) + { + configurationBuilder.Add(new ServiceFabricConfigurationSource(packageName, configPrefix)); + } + + return configurationBuilder; + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost.ServiceFabric/ServiceFabricConfigurationSource.cs b/Services/DataX.ServiceHost/DataX.ServiceHost.ServiceFabric/ServiceFabricConfigurationSource.cs new file mode 100644 index 000000000..3cec2eeea --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost.ServiceFabric/ServiceFabricConfigurationSource.cs @@ -0,0 +1,45 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +namespace DataX.ServiceHost.ServiceFabric +{ + using Microsoft.Extensions.Configuration.Memory; + using System.Collections.Generic; + using System.Fabric; + + /// + public class ServiceFabricConfigurationSource : MemoryConfigurationSource + { + private readonly string _configPrefix; + + public ServiceFabricConfigurationSource(string packageName, string configPrefix) + { + _configPrefix = configPrefix.EndsWith(":") ? configPrefix : configPrefix + ":"; + + var package = FabricRuntime.GetActivationContext().GetConfigurationPackageObject(packageName); + + this.InitialData = GetFlatConfig(package); + } + + /// + /// Creates a flat configuration compatible with IConfiguration from ServiceFabric's ConfigurationPackage + /// + /// The ConfigurationPackage to flatten + /// A KeyValuePair IEnumerable to be used for + private IEnumerable> GetFlatConfig(ConfigurationPackage package) + { + var flatConfig = new Dictionary(); + foreach (var section in package.Settings.Sections) + { + foreach (var parameter in section.Parameters) + { + flatConfig[$"{_configPrefix}{section.Name}:{parameter.Name}"] = parameter.Value; + } + } + + return flatConfig; + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost/Authorization/DataXAuthRequirement.cs b/Services/DataX.ServiceHost/DataX.ServiceHost/Authorization/DataXAuthRequirement.cs new file mode 100644 index 000000000..748e88290 --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost/Authorization/DataXAuthRequirement.cs @@ -0,0 +1,45 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +namespace DataX.ServiceHost.Authorization +{ + using DataX.Contract.Settings; + using Microsoft.AspNetCore.Authorization; + using System.Threading.Tasks; + + /// + /// For assertion requirements. We're extending this so that we can easily identify the DataX requirement instance + /// when adding in policy requirements. This lets us prevent duplication of requirements and handlers. + /// + public abstract class DataXAuthRequirement : IAuthorizationHandler, IAuthorizationRequirement + { + public DataXSettings Settings { get; set; } + + public DataXAuthRequirement() { } + + public DataXAuthRequirement(DataXSettings settings) + { + Settings = settings; + } + + /// + public Task HandleAsync(AuthorizationHandlerContext context) + { + if(IsAuthorized(context, Settings)) + { + context.Succeed(this); + } + + return Task.CompletedTask; + } + + /// + /// Given the auth context and settings, determines if a request is authorized. + /// + /// Provided AuthorizationHandlerContext + /// Provided DataXSettings + /// True if determined to be authorized, else false. + protected abstract bool IsAuthorized(AuthorizationHandlerContext context, DataXSettings settings); + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost/Authorization/Requirements/DataXOneBoxRequirement.cs b/Services/DataX.ServiceHost/DataX.ServiceHost/Authorization/Requirements/DataXOneBoxRequirement.cs new file mode 100644 index 000000000..1d708bf88 --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost/Authorization/Requirements/DataXOneBoxRequirement.cs @@ -0,0 +1,25 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.Contract.Settings; +using Microsoft.AspNetCore.Authorization; +using System; +using System.Collections.Generic; +using System.Text; + +namespace DataX.ServiceHost.Authorization.Requirements +{ + public class DataXOneBoxRequirement : DataXAuthRequirement + { + public DataXOneBoxRequirement() { } + + public DataXOneBoxRequirement(DataXSettings settings) + : base(settings) { } + + protected override bool IsAuthorized(AuthorizationHandlerContext context, DataXSettings settings) + { + return settings.EnableOneBox; + } + } +} diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost/DataX.ServiceHost.csproj b/Services/DataX.ServiceHost/DataX.ServiceHost/DataX.ServiceHost.csproj index 86ea3bbe7..c9bade699 100644 --- a/Services/DataX.ServiceHost/DataX.ServiceHost/DataX.ServiceHost.csproj +++ b/Services/DataX.ServiceHost/DataX.ServiceHost/DataX.ServiceHost.csproj @@ -1,7 +1,28 @@ - + - netcoreapp2.1 + netstandard2.0 + true + FinalPublicKey.snk + true + + + + all + runtime; build; native; contentfiles; analyzers + + + + + + + + + Microsoft400 + StrongName + + + diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost/FinalPublicKey.snk b/Services/DataX.ServiceHost/DataX.ServiceHost/FinalPublicKey.snk new file mode 100644 index 000000000..110b59c7b Binary files /dev/null and b/Services/DataX.ServiceHost/DataX.ServiceHost/FinalPublicKey.snk differ diff --git a/Services/DataX.ServiceHost/DataX.ServiceHost/HostUtil.cs b/Services/DataX.ServiceHost/DataX.ServiceHost/HostUtil.cs new file mode 100644 index 000000000..37ecda5e6 --- /dev/null +++ b/Services/DataX.ServiceHost/DataX.ServiceHost/HostUtil.cs @@ -0,0 +1,18 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using System; +using System.Collections.Generic; +using System.Text; + +namespace DataX.ServiceHost +{ + public static class HostUtil + { + /// + /// Returns true if Fabric_ApplicationName environment variable is not null, otherwise false if it is null. + /// + public static bool InServiceFabric => Environment.GetEnvironmentVariable("Fabric_ApplicationName") != null; + } +} diff --git a/Services/DataX.SimulatedData/DataX.SimulatedData.DataGenService/DataX.SimulatedData.DataGenService.csproj b/Services/DataX.SimulatedData/DataX.SimulatedData.DataGenService/DataX.SimulatedData.DataGenService.csproj index 5bed5e26f..06ea2b41a 100644 --- a/Services/DataX.SimulatedData/DataX.SimulatedData.DataGenService/DataX.SimulatedData.DataGenService.csproj +++ b/Services/DataX.SimulatedData/DataX.SimulatedData.DataGenService/DataX.SimulatedData.DataGenService.csproj @@ -2,7 +2,7 @@ Exe - netcoreapp2.1 + netcoreapp2.2 True win7-x64 obj @@ -40,14 +40,14 @@ - Microsoft + Microsoft400 StrongName - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.SimulatedData/DataX.SimulatedData.DataGenServiceTest/DataX.SimulatedData.DataGenServiceTest.csproj b/Services/DataX.SimulatedData/DataX.SimulatedData.DataGenServiceTest/DataX.SimulatedData.DataGenServiceTest.csproj index b9cdf305e..72f35dd62 100644 --- a/Services/DataX.SimulatedData/DataX.SimulatedData.DataGenServiceTest/DataX.SimulatedData.DataGenServiceTest.csproj +++ b/Services/DataX.SimulatedData/DataX.SimulatedData.DataGenServiceTest/DataX.SimulatedData.DataGenServiceTest.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 false @@ -55,7 +55,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.SimulatedData/DataXSimulatedData.nuspec b/Services/DataX.SimulatedData/DataXSimulatedData.nuspec index 2b74d06e4..95457f564 100644 --- a/Services/DataX.SimulatedData/DataXSimulatedData.nuspec +++ b/Services/DataX.SimulatedData/DataXSimulatedData.nuspec @@ -16,7 +16,7 @@ - + \ No newline at end of file diff --git a/Services/DataX.Utilities/DataX.Utilities.Blob/BlobHelper.cs b/Services/DataX.Utilities/DataX.Utilities.Blob/BlobHelper.cs index f8ecf030c..e543b6ffe 100644 --- a/Services/DataX.Utilities/DataX.Utilities.Blob/BlobHelper.cs +++ b/Services/DataX.Utilities/DataX.Utilities.Blob/BlobHelper.cs @@ -9,13 +9,18 @@ using System.Collections.Generic; using System.IO; using System.Linq; -using System.Text; using System.Threading.Tasks; +using System.Text.RegularExpressions; +using DataX.Contract.Exception; namespace DataX.Utilities.Blob { public static class BlobHelper { + /// + /// Get blob content + /// + /// public static string GetBlobContent(string connectionString, string blobUri) { var account = CloudStorageAccount.Parse(connectionString); @@ -45,6 +50,10 @@ public static async Task DeleteBlob(string connectionString, string b return ApiResult.CreateSuccess("Blob Deleted"); } + /// + /// Delete all blobs in a container + /// + /// public static async Task DeleteAllBlobsInAContainer(string storageConnectionString, string containerName, string blobDirectory) { CloudStorageAccount storageAccount = CloudStorageAccount.Parse(storageConnectionString); @@ -85,7 +94,10 @@ public static async Task DeleteAllBlobsInAContainer(string storageCon return ApiResult.CreateSuccess("Deleted Successfully"); } - + /// + /// Helper to read a blob content + /// + /// private static string ReadFromBlob(CloudBlockBlob blockBlob) { string text = null; @@ -107,12 +119,19 @@ private static string ReadFromBlob(CloudBlockBlob blockBlob) return text; } + /// + /// Save a content to a blob + /// public static void SaveContentToBlob(string connectionString, string blobUri, string content) { CloudBlockBlob blob = GetBlobReference(connectionString, blobUri); SaveToBlob(blob, content); } + /// + /// Get the contents of all blobs in a container + /// + /// public static Dictionary GetBlobsFromContainer(string connectionString, string containerName, string folderPath) { var account = CloudStorageAccount.Parse(connectionString); @@ -132,6 +151,10 @@ public static Dictionary GetBlobsFromContainer(string connection return blobContents; } + /// + /// Helper to get a blob reference + /// + /// private static CloudBlockBlob GetBlobReference(string connectionString, string blobUri) { var account = CloudStorageAccount.Parse(connectionString); @@ -140,10 +163,181 @@ private static CloudBlockBlob GetBlobReference(string connectionString, string b return new CloudBlockBlob(new Uri(blobUri), client.Credentials); } + /// + /// Helper to save a content to a blob + /// private static void SaveToBlob(CloudBlockBlob blockBlob, string content) { // TODO Refactor blockBlob.UploadTextAsync(content).Wait(); } + + /// + /// Get a content of a list of blobs which are sorted by last modified time + /// + /// + public static async Task> GetLastModifiedBlobContentsInBlobPath(string connectionString, string containerName, string prefix, string blobPathPattern, int blobCount) + { + CloudStorageAccount storageAccount = CloudStorageAccount.Parse(connectionString); + CloudBlobClient cloudBlobClient = storageAccount.CreateCloudBlobClient(); + CloudBlobContainer container = cloudBlobClient.GetContainerReference(containerName); + + var allBlobs = await container.ListBlobsSegmentedAsync(prefix: prefix, useFlatBlobListing: true, blobListingDetails: BlobListingDetails.None, maxResults: null, currentToken: null, options: null, operationContext: null).ConfigureAwait(false); + var filteredBlobs = allBlobs.Results.OfType().OrderByDescending(m => m.Properties.LastModified); + + List blobContents = new List(); + int queueCount = 0; + + foreach (CloudBlockBlob blob in filteredBlobs) + { + if (ValidateBlobPath(blobPathPattern, blob.Uri.ToString()) && blob.Properties.Length > 0) + { + using (var stream = await blob.OpenReadAsync().ConfigureAwait(false)) + { + using (var sr = new StreamReader(stream)) + { + int offset = 0; + while (blob.Properties.Length > offset) + { + // We can't read the entire blob if a blob is too big (1 > GB) + // So we just read it line by line until we have enough information (500 rows) to generate schema + var line = await sr.ReadLineAsync().ConfigureAwait(false); + if (!string.IsNullOrEmpty(line)) + { + if (ValidateJson(line)) + { + blobContents.Add(line); + + if (++queueCount >= blobCount) + { + return blobContents; + } + } + + offset += line.Length; + } + + if (sr.EndOfStream) + { + break; + } + } + } + } + } + } + + return blobContents; + } + + /// + /// Parse and get a prefix for a blob path + /// + /// + public static string ParsePrefix(string blobUri) + { + if (!Uri.TryCreate(blobUri, UriKind.Absolute, out var uri)) + { + return blobUri; + } + + var prefix = ""; + foreach (var seg in uri.Segments) + { + // It will keep adding every segment to compose a path prefix until we find the first segment which starts with "{". + // e.g. When an input is mycontainer@mysa.blob.core.windows.net/mypath/mypath2/mypath3/{yyyy}/{MM}/{dd}, the output is mypath/mypath2/mypath3/ + if (seg.StartsWith("%7B")) // "{", '\u007B' + { + break; + } + + prefix = Path.Combine(prefix, seg); + } + + return prefix.TrimStart('/'); + } + + /// + /// Get a regex pattern for a blob path + /// when an input is myoutputs@somesa.blob.core.windows.net/Test/{yyyy-MM-dd} + /// the output is somesa.blob.core.windows.net/myoutputs/Test/(\w+)-(\w+)-(\w+) + /// + /// + public static string GenerateRegexPatternFromPath(string path) + { + path = NormalizeBlobPath(path); + var mc = Regex.Matches(path, @"{(.*?)}"); + if (mc == null || mc.Count < 1) + { + return path; + } + + foreach (Match m in mc) + { + var r3 = Regex.Match(m.Value, @"^({)*([yMdHhmsS\-\/.,: ]+)(})*$"); + if (!r3.Success) + { + throw new GeneralException("Token in the blob path should be a data time format. e.g. {yyyy-MM-dd}"); + } + + path = path.Replace(m.Value, @"(\w+)", StringComparison.InvariantCulture); + } + + return path; + } + + /// + /// Normalize a blob path + /// when an input is myoutputs@somesa.blob.core.windows.net/Test/{yyyy-MM-dd} + /// the output is myoutputs@somesa.blob.core.windows.net/Test/{yyyy}-{MM}-{dd} + /// + /// + private static string NormalizeBlobPath(string path) + { + path = path.TrimEnd('/'); + var mc = Regex.Matches(path, @"{(.*?)}"); + if (mc == null || mc.Count < 1 || mc.Count > 1) + { + return path; + } + + var tokenValue = mc[0].Value.Trim(new char[] { '{', '}' }); + + var mc2 = Regex.Matches(tokenValue, @"[A-Za-z]+"); + foreach (Match m in mc2) + { + tokenValue = tokenValue.Replace(m.Value, "{" + m.Value + "}", StringComparison.InvariantCulture); + } + + path = path.Replace(mc[0].Value, tokenValue, StringComparison.InvariantCulture); + + return path; + } + + /// + /// Test if the input is a valid json + /// + /// + private static bool ValidateJson(string input) + { + try + { + Newtonsoft.Json.Linq.JObject.Parse(input); + return true; + } + catch + { + return false; + } + } + + /// + /// Test if the blob path matches the expected blob pattern + /// + /// + private static bool ValidateBlobPath(string blobPathPattern, string blobFullPath) + { + return Regex.Match(blobFullPath, blobPathPattern).Success; + } } } diff --git a/Services/DataX.Utilities/DataX.Utilities.Blob/DataX.Utilities.Blob.csproj b/Services/DataX.Utilities/DataX.Utilities.Blob/DataX.Utilities.Blob.csproj index 500fd6da3..0b545a30a 100644 --- a/Services/DataX.Utilities/DataX.Utilities.Blob/DataX.Utilities.Blob.csproj +++ b/Services/DataX.Utilities/DataX.Utilities.Blob/DataX.Utilities.Blob.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -36,7 +36,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Utilities/DataX.Utilities.CosmosDB/DataX.Utilities.CosmosDB.csproj b/Services/DataX.Utilities/DataX.Utilities.CosmosDB/DataX.Utilities.CosmosDB.csproj index edbefca79..dc587e21e 100644 --- a/Services/DataX.Utilities/DataX.Utilities.CosmosDB/DataX.Utilities.CosmosDB.csproj +++ b/Services/DataX.Utilities/DataX.Utilities.CosmosDB/DataX.Utilities.CosmosDB.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -31,7 +31,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Utilities/DataX.Utilities.EventHub/DataX.Utilities.EventHub.csproj b/Services/DataX.Utilities/DataX.Utilities.EventHub/DataX.Utilities.EventHub.csproj index 40c4ceb33..9cfa89507 100644 --- a/Services/DataX.Utilities/DataX.Utilities.EventHub/DataX.Utilities.EventHub.csproj +++ b/Services/DataX.Utilities/DataX.Utilities.EventHub/DataX.Utilities.EventHub.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -23,7 +23,7 @@ all runtime; build; native; contentfiles; analyzers - + all runtime; build; native; contentfiles; analyzers @@ -38,7 +38,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Utilities/DataX.Utilities.KeyVault/DataX.Utilities.KeyVault.csproj b/Services/DataX.Utilities/DataX.Utilities.KeyVault/DataX.Utilities.KeyVault.csproj index 1a5aba1e8..431725546 100644 --- a/Services/DataX.Utilities/DataX.Utilities.KeyVault/DataX.Utilities.KeyVault.csproj +++ b/Services/DataX.Utilities/DataX.Utilities.KeyVault/DataX.Utilities.KeyVault.csproj @@ -30,7 +30,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Utilities/DataX.Utilities.KeyVault/KeyVault.cs b/Services/DataX.Utilities/DataX.Utilities.KeyVault/KeyVault.cs index 7cc3ce3b3..fde1607bf 100644 --- a/Services/DataX.Utilities/DataX.Utilities.KeyVault/KeyVault.cs +++ b/Services/DataX.Utilities/DataX.Utilities.KeyVault/KeyVault.cs @@ -40,7 +40,7 @@ private static void ParseSecretUri(string secretUri, out string keyvaultName, ou { try { - Regex reg = new Regex(@"^((keyvault:?):\/\/)?([^:\/\s]+)(\/)(.*)?", RegexOptions.IgnoreCase); + Regex reg = new Regex(@"^((keyvault|secretscope:?):\/\/)?([^:\/\s]+)(\/)(.*)?", RegexOptions.IgnoreCase); MatchCollection m = reg.Matches(secretUri); keyvaultName = m[0].Groups[3].Value; secretName = m[0].Groups[5].Value; diff --git a/Services/DataX.Utilities/DataX.Utilities.Telemetry/DataX.Utilities.Telemetry.csproj b/Services/DataX.Utilities/DataX.Utilities.Telemetry/DataX.Utilities.Telemetry.csproj index dd142194f..8daba4963 100644 --- a/Services/DataX.Utilities/DataX.Utilities.Telemetry/DataX.Utilities.Telemetry.csproj +++ b/Services/DataX.Utilities/DataX.Utilities.Telemetry/DataX.Utilities.Telemetry.csproj @@ -1,7 +1,7 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -17,7 +17,7 @@ - + all runtime; build; native; contentfiles; analyzers @@ -31,19 +31,20 @@ + - Microsoft + Microsoft400 StrongName - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Utilities/DataX.Utilities.Telemetry/StartUpUtil.cs b/Services/DataX.Utilities/DataX.Utilities.Telemetry/StartUpUtil.cs index b1da9d0e7..1bd512a34 100644 --- a/Services/DataX.Utilities/DataX.Utilities.Telemetry/StartUpUtil.cs +++ b/Services/DataX.Utilities/DataX.Utilities.Telemetry/StartUpUtil.cs @@ -3,6 +3,7 @@ // Licensed under the MIT License // ********************************************************************* using DataX.ServiceHost.ServiceFabric; +using DataX.Contract.Settings; using Microsoft.ApplicationInsights.AspNetCore.Extensions; using Microsoft.ApplicationInsights.Extensibility; using Microsoft.AspNetCore.Builder; @@ -17,36 +18,54 @@ namespace DataX.Utilities.Telemetry { public static class StartUpUtil { - public static void ConfigureServices(IServiceCollection services) + public static void ConfigureServices(IServiceCollection services, IConfiguration configuration) { - services.AddSingleton(); - services.AddApplicationInsightsTelemetry(new ApplicationInsightsServiceOptions() - { - EnableAdaptiveSampling = false, - EnableDebugLogger = false, - InstrumentationKey = KeyVault.KeyVault.GetSecretFromKeyvault(ServiceFabricUtil.GetServiceKeyVaultName().Result.ToString(), ServiceFabricUtil.GetServiceFabricConfigSetting("AppInsightsIntrumentationKey").Result.ToString()) - }); - services.AddSingleton(); - services.AddLogging(logging => - { - try - { - // In order to log ILogger logs - logging.AddApplicationInsights(); - // Optional: Apply filters to configure LogLevel Information or above is sent to - // ApplicationInsights for all categories. - logging.AddFilter("", LogLevel.Information); + var settings = configuration.GetSection(DataXSettingsConstants.ServiceEnvironment).Get(); - // Additional filtering For category starting in "Microsoft", - // only Warning or above will be sent to Application Insights. - logging.AddFilter("Microsoft", LogLevel.Warning); + ConfigureServices(services, settings ?? new DataXSettings()); + } - } - catch (Exception e) + public static void ConfigureServices(IServiceCollection services, DataXSettings settings) + { + services + .AddSingleton() + .AddApplicationInsightsTelemetry(new ApplicationInsightsServiceOptions() { - ServiceEventSource.Current.Message($"ApplicationInsights Error: {e.Message}"); - } - }); + EnableAdaptiveSampling = false, + EnableDebugLogger = false, + InstrumentationKey = GetInstrumentationKey(settings) + }) + .AddSingleton() + .AddLogging(logging => + { + try + { + // In order to log ILogger logs + logging.AddApplicationInsights(); + // Optional: Apply filters to configure LogLevel Information or above is sent to + // ApplicationInsights for all categories. + logging.AddFilter("", LogLevel.Information); + + // Additional filtering For category starting in "Microsoft", + // only Warning or above will be sent to Application Insights. + logging.AddFilter("Microsoft", LogLevel.Warning); + + } + catch (Exception e) + { + ServiceEventSource.Current.Message($"ApplicationInsights Error: {e.Message}"); + } + }); + } + + private static string GetInstrumentationKey(DataXSettings settings) + { + var secretName = settings?.AppInsightsIntrumentationKey; + var vaultName = settings.ServiceKeyVaultName; + + return string.IsNullOrWhiteSpace(secretName) || string.IsNullOrWhiteSpace(vaultName) + ? Guid.Empty.ToString() + : KeyVault.KeyVault.GetSecretFromKeyvault(settings.ServiceKeyVaultName, settings.AppInsightsIntrumentationKey); } } } diff --git a/Services/DataX.Utilities/DataX.Utilities.Web/DataX.Utilities.Web.csproj b/Services/DataX.Utilities/DataX.Utilities.Web/DataX.Utilities.Web.csproj index 71469d881..b53a74850 100644 --- a/Services/DataX.Utilities/DataX.Utilities.Web/DataX.Utilities.Web.csproj +++ b/Services/DataX.Utilities/DataX.Utilities.Web/DataX.Utilities.Web.csproj @@ -1,14 +1,14 @@  - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true - + all runtime; build; native; contentfiles; analyzers @@ -24,13 +24,14 @@ - + + - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Utilities/DataX.Utilities.Web/RolesCheck.cs b/Services/DataX.Utilities/DataX.Utilities.Web/RolesCheck.cs index a8d66ddb9..79e769431 100644 --- a/Services/DataX.Utilities/DataX.Utilities.Web/RolesCheck.cs +++ b/Services/DataX.Utilities/DataX.Utilities.Web/RolesCheck.cs @@ -6,14 +6,21 @@ using DataX.Contract; using System; using DataX.Gateway.Contract; - +using DataX.ServiceHost; +using DataX.ServiceHost.ServiceFabric; +using System.Collections.Generic; +using System.Fabric; +using System.Linq; + namespace DataX.Utilities.Web { public static class RolesCheck { // Default role names for reader/writer. These are made public so that service can override these values. public static string ReaderRoleName { get; set; } = "DataXReader"; - public static string WriterRoleName { get; set; } = "DataXWriter"; + public static string WriterRoleName { get; set; } = "DataXWriter"; + + private static readonly HashSet _ClientWhitelist = new HashSet(); public static void EnsureWriter(HttpRequest request, bool isLocal) { @@ -24,12 +31,51 @@ public static void EnsureWriter(HttpRequest request, bool isLocal) } } + /// + /// A helper method that Adds the test client user id to the white list from keyvault if it exists + /// TODO: Support adding this whitelist on Kubernetes using IConfiguration object + /// + private static void AddWhitelistedTestClientUserId() + { + if (HostUtil.InServiceFabric) + { + var serviceKeyvaultName = ServiceFabricUtil.GetServiceKeyVaultName().Result.ToString(); + var configPackage = FabricRuntime.GetActivationContext().GetConfigurationPackageObject("Config"); + var serviceEnvironmenConfig = configPackage.Settings.Sections["ServiceEnvironment"]; + var testClientId = serviceEnvironmenConfig.Parameters["TestClientId"].Value; + try + { + // Each secret needs to be in the format {ObjectId}.{TenantId} + List userIdList = KeyVault.KeyVault.GetSecretFromKeyvault(serviceKeyvaultName, testClientId).Split(new char[] { ',' }).ToList(); + foreach (string userId in userIdList) + { + _ClientWhitelist.Add(userId); + } + } + catch(Exception e) + { + // Do nothing in case the secret does not exist. + var message = e.Message; + } + } + } + public static void EnsureWriter(HttpRequest request) - { + { + // Ensure* methods only work when auth is handled at the Gateway in Service Fabric setup + // Otherwise ASP.NET Core is used and does not require this check + if(!HostUtil.InServiceFabric) + { + return; + } + var userrole = request.Headers[Constants.UserRolesHeader]; + var userID = request.Headers[Constants.UserIdHeader]; + AddWhitelistedTestClientUserId(); + Ensure.NotNull(userrole, "userrole"); - if (!userrole.ToString().Contains(WriterRoleName)) + if (!userrole.ToString().Contains(WriterRoleName) && !_ClientWhitelist.Contains(userID)) { throw new System.Exception($"{WriterRoleName} role needed to perform this action. User has the following roles: {userrole}"); } @@ -46,10 +92,19 @@ public static void EnsureReader(HttpRequest request, bool isLocal) public static void EnsureReader(HttpRequest request) { + // Ensure* methods only work when auth is handled at the Gateway in Service Fabric setup + // Otherwise ASP.NET Core is used and does not require this check + if (!HostUtil.InServiceFabric) + { + return; + } + var userrole = request.Headers[Constants.UserRolesHeader]; + var userID = request.Headers[Constants.UserIdHeader]; + AddWhitelistedTestClientUserId(); Ensure.NotNull(userrole, "userrole"); - if (!userrole.ToString().Contains(ReaderRoleName) && !userrole.ToString().Contains(WriterRoleName)) + if (!userrole.ToString().Contains(ReaderRoleName) && !userrole.ToString().Contains(WriterRoleName) && !_ClientWhitelist.Contains(userID)) { throw new System.Exception($"{ReaderRoleName} role needed to perform this action. User has the following roles: {userrole}"); } diff --git a/Services/DataX.Utilities/DataX.Utility.Blob/BaseTableEntity.cs b/Services/DataX.Utilities/DataX.Utility.Blob/BaseTableEntity.cs new file mode 100644 index 000000000..5bd563699 --- /dev/null +++ b/Services/DataX.Utilities/DataX.Utility.Blob/BaseTableEntity.cs @@ -0,0 +1,48 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using Microsoft.Azure.Cosmos.Table; +using System; +using System.Collections.Generic; + +namespace DataX.Utilities +{ + /// + /// CosmosTable base class + /// + public class BaseTableEntity : TableEntity + { + // azure storage limits + private const long _MAX_FIELD_LENGTH = 65536; // 64KB + private const long _MAX_TOTAL_FIELD_LENGTH = 1048576; // 1MB + + public override IDictionary WriteEntity(OperationContext operationContext) + { + + var properties = base.WriteEntity(operationContext); + + var totalLength = 0; + foreach (var property in properties) + { + if (property.Value.PropertyType == EdmType.String && property.Value.StringValue != null) + { + var value = property.Value.StringValue; + totalLength += value.Length; + if (value.Length > _MAX_FIELD_LENGTH) + { + var e = new ArgumentException("The maximum length of a single field (64KB) has been exceeded", property.Key); + throw e; + } + else if (totalLength > _MAX_TOTAL_FIELD_LENGTH) + { + throw new ArgumentException("The maximum length of all fields (1MB) has been exceeded"); + } + } + } + + return properties; + } + } +} diff --git a/Services/DataX.Utilities/DataX.Utility.Blob/BlobUtility.cs b/Services/DataX.Utilities/DataX.Utility.Blob/BlobUtility.cs index 3cd7c5453..8a7acad6b 100644 --- a/Services/DataX.Utilities/DataX.Utility.Blob/BlobUtility.cs +++ b/Services/DataX.Utilities/DataX.Utility.Blob/BlobUtility.cs @@ -61,7 +61,7 @@ public static async Task DeleteBlob(string connectionString, string blobUr public static async Task DeleteAllBlobsInAContainer(string storageConnectionString, string containerName, string directory) { CloudStorageAccount storageAccount = CloudStorageAccount.Parse(storageConnectionString); - if (storageAccount != null) + if (storageAccount == null) { return true; } diff --git a/Services/DataX.Utilities/DataX.Utility.Blob/DataX.Utility.Blob.csproj b/Services/DataX.Utilities/DataX.Utility.Blob/DataX.Utility.Blob.csproj index 8ff276404..16af752fe 100644 --- a/Services/DataX.Utilities/DataX.Utility.Blob/DataX.Utility.Blob.csproj +++ b/Services/DataX.Utilities/DataX.Utility.Blob/DataX.Utility.Blob.csproj @@ -1,7 +1,7 @@ - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -20,6 +20,9 @@ + + + all runtime; build; native; contentfiles; analyzers @@ -33,9 +36,13 @@ - Microsoft + Microsoft400 StrongName + + + + diff --git a/Services/DataX.Utilities/DataX.Utility.Blob/IsCriticalException.cs b/Services/DataX.Utilities/DataX.Utility.Blob/IsCriticalException.cs new file mode 100644 index 000000000..84b8913af --- /dev/null +++ b/Services/DataX.Utilities/DataX.Utility.Blob/IsCriticalException.cs @@ -0,0 +1,24 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using System; +using System.Threading; + +namespace DataX.Utilities +{ + public static class ExceptionExtensions + { + /// + /// Returns true if an exception should not be handled by logging code. + /// + public static bool IsCriticalException(this Exception ex) + { + return ex is StackOverflowException || + ex is OutOfMemoryException || + ex is ThreadAbortException || + ex is AccessViolationException; + } + } +} diff --git a/Services/DataX.Utilities/DataX.Utility.Blob/StorageTableBase.cs b/Services/DataX.Utilities/DataX.Utility.Blob/StorageTableBase.cs new file mode 100644 index 000000000..b113b9696 --- /dev/null +++ b/Services/DataX.Utilities/DataX.Utility.Blob/StorageTableBase.cs @@ -0,0 +1,81 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using DataX.Contract; +using Microsoft.Azure.Cosmos.Table; +using System.Collections.Generic; +using System.Threading.Tasks; + +namespace DataX.Utilities.Storage +{ + /// + /// Utility base class for CosmosTable + /// + /// + public abstract class StorageTableBase where TEntity: TableEntity, new() + { + public StorageTableBase(CloudTable table) + { + this.Table = table; + } + + protected CloudTable Table { get; } + + protected async Task RetrieveAsync(string partitionKey, string rowKey) + { + if (string.IsNullOrWhiteSpace(partitionKey) || string.IsNullOrWhiteSpace(rowKey)) + { + return null; + } + + return await this.Table.ExecuteAndGetResultAsync(TableOperation.Retrieve(partitionKey, rowKey)) + .ContinueOnAnyContext(); + } + + public async Task CreateWithRetryAsync(TEntity entity) + { + Ensure.NotNull(entity, nameof(entity)); + + return await StorageUtils.RetryOnConflictAsync( + async () => + { + + await this.Table.InsertAsync(entity); + return entity; + }); + } + + public async Task> QueryAsync() + { + return await this.Table.CreateQuery().ExecuteQueryAsync(); + } + + public async Task> RetrieveAllAsync() + { + var allEntities = new List(); + return await this.Table.ExecuteQueryAsync(new TableQuery()); + } + + public Task InsertOrReplace(TEntity entity) + { + return this.Table.InsertOrReplace(entity); + } + + public Task InsertAsync(TEntity entity) + { + return this.Table.InsertAsync(entity); + } + + public Task ReplaceAsync(TEntity entity) + { + return this.Table.ReplaceAsync(entity); + } + + public Task DeleteAsync(TEntity entity) + { + return this.Table.DeleteAsync(entity); + } + } +} diff --git a/Services/DataX.Utilities/DataX.Utility.Blob/StorageUtils.cs b/Services/DataX.Utilities/DataX.Utility.Blob/StorageUtils.cs new file mode 100644 index 000000000..dcc9c9c5d --- /dev/null +++ b/Services/DataX.Utilities/DataX.Utility.Blob/StorageUtils.cs @@ -0,0 +1,129 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using Microsoft.Azure.Cosmos.Table; +using Microsoft.Azure.Storage.Blob; +using Microsoft.Azure.Storage.File; +using System; +using System.Diagnostics; +using System.Net; +using System.Threading.Tasks; + +namespace DataX.Utilities.Storage +{ + public static class StorageUtils + { + /// + /// Calls an async function, and if it fails due to a table storage precondition + /// failure will attempt to retry the operation. + /// + /// The function to be retried on failure + /// + /// An optional notification that only gets called on a conflict. + /// Return true if after running the conflict resolution task we need + /// to retry. Return false if retry is not necessary. + /// + public static async Task RetryOnConflictAsync(Func a, Func> conflict = null) + { + while (true) + { + try + { + await a().ContinueOnAnyContext(); + break; + } + catch (Microsoft.Azure.Cosmos.Table.StorageException se) + { + if (se.RequestInformation.HttpStatusCode != (int)HttpStatusCode.PreconditionFailed) + { + throw; + } + if (conflict != null) + { + if (!await conflict.Invoke().ContinueOnAnyContext()) + { + return; + } + } + } + } + } + public static async Task RetryOnConflictAsync(Func> a, Func> conflict = null) + { + while (true) + { + try + { + return await a().ContinueOnAnyContext(); + } + catch (Microsoft.Azure.Cosmos.Table.StorageException se) + { + if (se.RequestInformation.HttpStatusCode != (int)HttpStatusCode.PreconditionFailed) + { + throw; + } + if (conflict != null) + { + if (!await conflict.Invoke().ContinueOnAnyContext()) + { + return default(T); + } + } + } + } + } + + + + public static async Task CreateIfNotExistsSafeAsync(this CloudBlobContainer container) + { + return await DynamicCreateIfNotExistsSafeAsync(container).ContinueOnAnyContext(); + } + + public static async Task CreateIfNotExistsSafeAsync(this CloudTable cloudTable) + { + return await DynamicCreateIfNotExistsSafeAsync(cloudTable).ContinueOnAnyContext(); + } + + public static async Task CreateIfNotExistsSafeAsync(this CloudFileShare cloudFileShare) + { + return await DynamicCreateIfNotExistsSafeAsync(cloudFileShare).ContinueOnAnyContext(); + } + + /// + /// Wraps CreateIfNotExistsAsync in a backoff timer. Could take 2 + /// minutes of waiting until safe to recreate. This handles an issue + /// where the table in a middle of a deletion will throw. + /// + /// dynamic for storage types with a CreateIfNotExistsAsync method + /// True if created, False otherwise + private static async Task DynamicCreateIfNotExistsSafeAsync(dynamic storageType) + { + var succeeded = false; + var result = false; + StorageException last_exc = null; + for (int tries = 0; tries < 15; tries++) + { + try + { + // create if not exist can fail if it is in the process of being deleted. If we receive a 409 conflict, we should sleep and try again. + result = await storageType.CreateIfNotExistsAsync().ConfigureAwait(continueOnCapturedContext: false); + succeeded = true; + } + catch (StorageException e) when (e.Message == "The remote server returned an error: (409) Conflict.") + { + Debug.Write("While trying to create we hit a conflict. Likely a deletion of this is in progress"); + await Task.Delay(TimeSpan.FromSeconds(1 * tries)).ContinueOnAnyContext(); + last_exc = e; + } + if (succeeded) + { + return result; + } + } + throw last_exc; + } + } +} diff --git a/Services/DataX.Utilities/DataX.Utility.Blob/TableExtensions.cs b/Services/DataX.Utilities/DataX.Utility.Blob/TableExtensions.cs new file mode 100644 index 000000000..cae842888 --- /dev/null +++ b/Services/DataX.Utilities/DataX.Utility.Blob/TableExtensions.cs @@ -0,0 +1,166 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Microsoft.Azure.Cosmos.Table; +using System; +using System.Collections.Generic; +using System.Threading; +using System.Threading.Tasks; + +namespace DataX.Utilities.Storage +{ + /// + /// CosmosTable utility class + /// + public static class TableExtensions + { + public static async Task ExecuteAndGetResultAsync(this CloudTable table, TableOperation operation) + { + var result = (T)(await table.ExecuteAsync(operation).ContinueOnAnyContext()).Result; + return result; + } + + public static async Task> ExecuteQueryAsync(this CloudTable table, TableQuery query, CancellationToken ct = default(CancellationToken)) where T : ITableEntity, new() + { + var items = new List(); + TableContinuationToken token = null; + + do + { + TableQuerySegment seg = await table.ExecuteQuerySegmentedAsync(query, token).ContinueOnAnyContext(); + token = seg.ContinuationToken; + items.AddRange(seg); + } while (token != null && !ct.IsCancellationRequested && (query.TakeCount == null || items.Count < query.TakeCount.Value)); + + return items; + } + + public static async Task ForEachQueryResultAsync(this CloudTable table, TableQuery query, Action processResult, CancellationToken ct = default(CancellationToken)) where T : ITableEntity, new() + { + TableContinuationToken token = null; + int count = 0; + do + { + TableQuerySegment seg = await table.ExecuteQuerySegmentedAsync(query, token).ContinueOnAnyContext(); + token = seg.ContinuationToken; + foreach (var result in seg) + { + processResult(result); + count++; + } + } while (token != null && !ct.IsCancellationRequested && (query.TakeCount == null || count < query.TakeCount.Value)); + } + + public static async Task ForEachQueryResultAsync(this CloudTable table, TableQuery query, Func processResultAsync, CancellationToken ct = default(CancellationToken)) where T : ITableEntity, new() + { + TableContinuationToken token = null; + int count = 0; + do + { + TableQuerySegment seg = await table.ExecuteQuerySegmentedAsync(query, token).ContinueOnAnyContext(); + token = seg.ContinuationToken; + foreach (var result in seg) + { + await processResultAsync(result).ContinueOnAnyContext(); + count++; + } + } while (token != null && !ct.IsCancellationRequested && (query.TakeCount == null || count < query.TakeCount.Value)); + } + + public static async Task ForEachQueryResultAsync(this CloudTable table, TableQuery query, TableContinuationToken token, Func processResultAsync, CancellationToken ct = default(CancellationToken)) where T : ITableEntity, new() + { + int count = 0; + do + { + TableQuerySegment seg = await table.ExecuteQuerySegmentedAsync(query, token).ContinueOnAnyContext(); + token = seg.ContinuationToken; + foreach (var result in seg) + { + await processResultAsync(result).ContinueOnAnyContext(); + count++; + } + } while (token != null && !ct.IsCancellationRequested && (query.TakeCount == null || count < query.TakeCount.Value)); + return token; + } + + public static async Task> ExecuteQueryAsync(this TableQuery query, CancellationToken ct = default(CancellationToken)) where T : ITableEntity, new() + { + var items = new List(); + TableContinuationToken token = null; + + do + { + TableQuerySegment seg = await query.ExecuteSegmentedAsync(token).ContinueOnAnyContext(); + token = seg.ContinuationToken; + items.AddRange(seg); + } while (token != null && !ct.IsCancellationRequested && (query.TakeCount == null || items.Count < query.TakeCount.Value)); + + return items; + } + + /// + /// Executes a table query and processes the results as they arrive + /// + public static async Task ForEachQueryResultAsync(this TableQuery query, Func processResultAsync, CancellationToken ct = default(CancellationToken)) where T : ITableEntity, new() + { + TableContinuationToken token = null; + + int count = 0; + do + { + TableQuerySegment seg = await query.ExecuteSegmentedAsync(token).ContinueOnAnyContext(); + token = seg.ContinuationToken; + foreach (var result in seg) + { + await processResultAsync(result).ContinueOnAnyContext(); + count++; + } + } while (token != null && !ct.IsCancellationRequested && (query.TakeCount == null || count < query.TakeCount.Value)); + } + + + /// + /// Executes a table query and processes the results as they arrive + /// + public static async Task ForEachQueryResultAsync(this TableQuery query, Func> processResultAsync, CancellationToken ct = default(CancellationToken)) where T : ITableEntity, new() + { + TableContinuationToken token = null; + + int count = 0; + do + { + TableQuerySegment seg = await query.ExecuteSegmentedAsync(token).ContinueOnAnyContext(); + token = seg.ContinuationToken; + foreach (var result in seg) + { + if (!await processResultAsync(result).ContinueOnAnyContext()) + { + break; + } + count++; + } + } while (token != null && !ct.IsCancellationRequested && (query.TakeCount == null || count < query.TakeCount.Value)); + } + + public static async Task InsertAsync(this CloudTable table, T entity) where T : TableEntity + { + return (T)(await table.ExecuteAsync(TableOperation.Insert(entity)).ContinueOnAnyContext()).Result; + } + + public static async Task InsertOrReplace(this CloudTable table, T entity) where T : TableEntity + { + return (T)(await table.ExecuteAsync(TableOperation.InsertOrReplace(entity)).ContinueOnAnyContext()).Result; + } + + public static Task DeleteAsync(this CloudTable table, TableEntity entity) + { + return table.ExecuteAsync(TableOperation.Delete(entity)); + } + + public static async Task ReplaceAsync(this CloudTable table, T entity) where T : TableEntity + { + return (T)(await table.ExecuteAsync(TableOperation.Replace(entity)).ContinueOnAnyContext()).Result; + } + } +} diff --git a/Services/DataX.Utilities/DataX.Utility.Blob/TaskExtensions.cs b/Services/DataX.Utilities/DataX.Utility.Blob/TaskExtensions.cs new file mode 100644 index 000000000..a96586918 --- /dev/null +++ b/Services/DataX.Utilities/DataX.Utility.Blob/TaskExtensions.cs @@ -0,0 +1,167 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using System; +using System.Diagnostics; +using System.Globalization; +using System.Runtime.CompilerServices; +using System.Runtime.ExceptionServices; +using System.Threading.Tasks; + +namespace DataX.Utilities.Storage +{ + public static class TaskExtensions + { + /// + /// Returns an awaitable which is configured to continue on any context + /// + public static ConfiguredTaskAwaitable ContinueOnAnyContext(this Task task) + { + return task.ConfigureAwait(continueOnCapturedContext: false); + } + + /// + /// Returns an awaitable which will is configured to continue on any context + /// + public static ConfiguredTaskAwaitable ContinueOnAnyContext(this Task task) + { + return task.ConfigureAwait(continueOnCapturedContext: false); + } + + /// + /// Synchronously waits on a task and returns its result. If the task threw an exception, this unwraps it and rethrows it preserving the original stack trace + /// + public static T SyncResultOrException(this Task task) + { + try + { + return task.Result; + } + catch (AggregateException ae) + { + var innerException = ae.Flatten().InnerException; + ExceptionDispatchInfo.Capture(innerException).Throw(); + return default(T); // Unreachable + } + } + + /// + /// Synchronously waits on a task. If the task threw an exception, this unwraps it and rethrows it preserving the original stack trace + /// + public static void SyncWaitOrException(this Task task) + { + try + { + task.Wait(); + } + catch (AggregateException ae) + { + var innerException = ae.Flatten().InnerException; + ExceptionDispatchInfo.Capture(innerException).Throw(); + } + } + + /// + /// Suppresses warnings about unawaited tasks and ensures that unhandled + /// errors will cause the process to terminate. + /// + public static async void DoNotWait(this Task task) + { + await task.ContinueOnAnyContext(); + } + + /// + /// Logs all exceptions from a task except those that return true from + /// , which are rethrown. + /// + public static async Task HandleAllExceptions( + this Task task, + Type callerType = null, + [CallerFilePath] string callerFile = null, + [CallerLineNumber] int callerLineNumber = 0, + [CallerMemberName] string callerName = null + ) + { + try + { + await task.ContinueOnAnyContext(); + } + catch (Exception ex) + { + if (ex.IsCriticalException()) + { + throw; + } + + var message = GetUnhandledExceptionString(ex, callerType, callerFile, callerLineNumber, callerName); + // Send the message to the trace listener in case there is + // somebody out there listening. + Trace.TraceError(message); + } + } + + /// + /// Retries the given task exponentially. + /// + public static async Task RetryExponentially(this Task task, bool retryOnException = true, int maxRetryAttempts = 5) + { + var retries = 1; + while (true) + { + try + { + await task.ContinueOnAnyContext(); + break; + } + catch (Exception ex) + { + if (retries > maxRetryAttempts) + { + throw new RetryExceededException($"Failure to complete task in specified retries {maxRetryAttempts}", ex); + } + + if (!retryOnException) + { + throw; + } + } + + await Task.Delay(1000 * retries); + retries++; + } + } + + public class RetryExceededException : Exception + { + public RetryExceededException() : base() { } + + public RetryExceededException(string message) : base(message) { } + + public RetryExceededException(string message, Exception innerEx) : base(message, innerEx) { } + } + + private static string GetUnhandledExceptionString( + Exception ex, + Type callerType, + [CallerFilePath] string callerFile = null, + [CallerLineNumber] int callerLineNumber = 0, + [CallerMemberName] string callerName = null + ) + { + if (string.IsNullOrEmpty(callerName)) + { + callerName = callerType != null ? callerType.FullName : string.Empty; + } + else if (callerType != null) + { + callerName = callerType.FullName + "." + callerName; + } + return string.Format(CultureInfo.CurrentUICulture, + @"Unhandled exception in {3} ({1}:{2}) +{0}", + ex, callerFile ?? string.Empty, callerLineNumber, callerName); + } + + } +} diff --git a/Services/DataX.Utilities/DataX.Utility.CosmosDB/DataX.Utility.CosmosDB.csproj b/Services/DataX.Utilities/DataX.Utility.CosmosDB/DataX.Utility.CosmosDB.csproj index e22e35cf9..1050087c5 100644 --- a/Services/DataX.Utilities/DataX.Utility.CosmosDB/DataX.Utility.CosmosDB.csproj +++ b/Services/DataX.Utilities/DataX.Utility.CosmosDB/DataX.Utility.CosmosDB.csproj @@ -1,7 +1,7 @@ - netcoreapp2.1 + netcoreapp2.2 true FinalPublicKey.snk true @@ -34,7 +34,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Utilities/DataX.Utility.KeyVault/DataX.Utility.KeyVault.csproj b/Services/DataX.Utilities/DataX.Utility.KeyVault/DataX.Utility.KeyVault.csproj index b6f9a466f..e7792a127 100644 --- a/Services/DataX.Utilities/DataX.Utility.KeyVault/DataX.Utility.KeyVault.csproj +++ b/Services/DataX.Utilities/DataX.Utility.KeyVault/DataX.Utility.KeyVault.csproj @@ -39,7 +39,7 @@ - Microsoft + Microsoft400 StrongName diff --git a/Services/DataX.Utilities/DataX.Utility.KeyVault/SecretUriParser.cs b/Services/DataX.Utilities/DataX.Utility.KeyVault/SecretUriParser.cs index 7431ffd3d..6a8cdcbb4 100644 --- a/Services/DataX.Utilities/DataX.Utility.KeyVault/SecretUriParser.cs +++ b/Services/DataX.Utilities/DataX.Utility.KeyVault/SecretUriParser.cs @@ -11,7 +11,7 @@ namespace DataX.Utility.KeyVault { public static class SecretUriParser { - private static Regex _Reg = new Regex(@"^((keyvault:?):\/\/)?([^:\/\s]+)(\/)(.*)?", RegexOptions.IgnoreCase); + private static Regex _Reg = new Regex(@"^((keyvault|secretscope:?):\/\/)?([^:\/\s]+)(\/)(.*)?", RegexOptions.IgnoreCase); public static void ParseSecretUri(string secretUri, out string keyvaultName, out string secretName) { @@ -45,9 +45,9 @@ public static bool TryParseSecretUri(string secretUri, out string keyvaultName, } } - public static string ComposeUri(string keyvaultName, string secretName) + public static string ComposeUri(string keyvaultName, string secretName, string uriPrefix) { - return $"keyvault://{keyvaultName}/{secretName}"; + return $"{uriPrefix}://{keyvaultName}/{secretName}"; } } } diff --git a/Services/DataX.Utilities/DataX.Utility.Nuget/DataX.Utility.Nuget.csproj b/Services/DataX.Utilities/DataX.Utility.Nuget/DataX.Utility.Nuget.csproj index 23fdae104..ff7833c5d 100644 --- a/Services/DataX.Utilities/DataX.Utility.Nuget/DataX.Utility.Nuget.csproj +++ b/Services/DataX.Utilities/DataX.Utility.Nuget/DataX.Utility.Nuget.csproj @@ -2,7 +2,7 @@ Library - netcoreapp2.1 + netcoreapp2.2 true diff --git a/Services/DataX.sln b/Services/DataX.sln index 387a0b446..6d3a2cf60 100644 --- a/Services/DataX.sln +++ b/Services/DataX.sln @@ -1,7 +1,7 @@  Microsoft Visual Studio Solution File, Format Version 12.00 # Visual Studio 15 -VisualStudioVersion = 15.0.28307.271 +VisualStudioVersion = 15.0.28307.645 MinimumVisualStudioVersion = 10.0.40219.1 Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Contract", "DataX.Contract\DataX.Contract.csproj", "{0F0A8226-E416-4796-B6BB-817B14E25AE4}" EndProject @@ -109,6 +109,8 @@ Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.Local.Test", " EndProject Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.Extensions", "DataX.Config\DataX.Utility.Extensions\DataX.Utility.Extensions.csproj", "{D5D3DD2D-760A-4818-A74F-C0061719841C}" EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Config.DatabricksClient", "DataX.Config\DataX.Config.DatabricksClient\DataX.Config.DatabricksClient.csproj", "{128BBA82-876A-461F-BEE4-5637CEFD54FA}" +EndProject Global GlobalSection(SolutionConfigurationPlatforms) = preSolution Debug|Any CPU = Debug|Any CPU @@ -467,14 +469,6 @@ Global {20E1DF46-301A-4F74-8E76-D29192E368E5}.Release|Any CPU.Build.0 = Release|Any CPU {20E1DF46-301A-4F74-8E76-D29192E368E5}.Release|x64.ActiveCfg = Release|Any CPU {20E1DF46-301A-4F74-8E76-D29192E368E5}.Release|x64.Build.0 = Release|Any CPU - {F2EAE040-4DC1-4409-8079-B7AABDCBF718}.Debug|Any CPU.ActiveCfg = Debug|Any CPU - {F2EAE040-4DC1-4409-8079-B7AABDCBF718}.Debug|Any CPU.Build.0 = Debug|Any CPU - {F2EAE040-4DC1-4409-8079-B7AABDCBF718}.Debug|x64.ActiveCfg = Debug|Any CPU - {F2EAE040-4DC1-4409-8079-B7AABDCBF718}.Debug|x64.Build.0 = Debug|Any CPU - {F2EAE040-4DC1-4409-8079-B7AABDCBF718}.Release|Any CPU.ActiveCfg = Release|Any CPU - {F2EAE040-4DC1-4409-8079-B7AABDCBF718}.Release|Any CPU.Build.0 = Release|Any CPU - {F2EAE040-4DC1-4409-8079-B7AABDCBF718}.Release|x64.ActiveCfg = Release|Any CPU - {F2EAE040-4DC1-4409-8079-B7AABDCBF718}.Release|x64.Build.0 = Release|Any CPU {C1CAC657-0322-4275-AE64-EC83FDA8C820}.Debug|Any CPU.ActiveCfg = Debug|Any CPU {C1CAC657-0322-4275-AE64-EC83FDA8C820}.Debug|Any CPU.Build.0 = Debug|Any CPU {C1CAC657-0322-4275-AE64-EC83FDA8C820}.Debug|x64.ActiveCfg = Debug|Any CPU @@ -499,6 +493,14 @@ Global {D5D3DD2D-760A-4818-A74F-C0061719841C}.Release|Any CPU.Build.0 = Release|Any CPU {D5D3DD2D-760A-4818-A74F-C0061719841C}.Release|x64.ActiveCfg = Release|Any CPU {D5D3DD2D-760A-4818-A74F-C0061719841C}.Release|x64.Build.0 = Release|Any CPU + {128BBA82-876A-461F-BEE4-5637CEFD54FA}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {128BBA82-876A-461F-BEE4-5637CEFD54FA}.Debug|Any CPU.Build.0 = Debug|Any CPU + {128BBA82-876A-461F-BEE4-5637CEFD54FA}.Debug|x64.ActiveCfg = Debug|Any CPU + {128BBA82-876A-461F-BEE4-5637CEFD54FA}.Debug|x64.Build.0 = Debug|Any CPU + {128BBA82-876A-461F-BEE4-5637CEFD54FA}.Release|Any CPU.ActiveCfg = Release|Any CPU + {128BBA82-876A-461F-BEE4-5637CEFD54FA}.Release|Any CPU.Build.0 = Release|Any CPU + {128BBA82-876A-461F-BEE4-5637CEFD54FA}.Release|x64.ActiveCfg = Release|Any CPU + {128BBA82-876A-461F-BEE4-5637CEFD54FA}.Release|x64.Build.0 = Release|Any CPU EndGlobalSection GlobalSection(SolutionProperties) = preSolution HideSolutionNode = FALSE @@ -547,7 +549,6 @@ Global {702A1D3B-63AF-4519-8334-600E477D9CBC} = {6FF7998A-7952-444B-AD85-E3A9EB8683C1} {13288F78-76B4-4B9F-B57B-A7415FEEF42F} = {6FF7998A-7952-444B-AD85-E3A9EB8683C1} {20E1DF46-301A-4F74-8E76-D29192E368E5} = {6FF7998A-7952-444B-AD85-E3A9EB8683C1} - {F2EAE040-4DC1-4409-8079-B7AABDCBF718} = {6FF7998A-7952-444B-AD85-E3A9EB8683C1} {C1CAC657-0322-4275-AE64-EC83FDA8C820} = {6FF7998A-7952-444B-AD85-E3A9EB8683C1} {9E338FCC-793F-4048-9672-048270C856C4} = {6FF7998A-7952-444B-AD85-E3A9EB8683C1} {D5D3DD2D-760A-4818-A74F-C0061719841C} = {6FF7998A-7952-444B-AD85-E3A9EB8683C1} @@ -555,4 +556,4 @@ Global GlobalSection(ExtensibilityGlobals) = postSolution SolutionGuid = {FB461262-8931-453E-8214-506BF892338E} EndGlobalSection -EndGlobal +EndGlobal \ No newline at end of file diff --git a/Services/JobRunner/AppConfig.cs b/Services/JobRunner/AppConfig.cs new file mode 100644 index 000000000..d229e66e8 --- /dev/null +++ b/Services/JobRunner/AppConfig.cs @@ -0,0 +1,47 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using Microsoft.Extensions.Configuration; +using System.Composition; + +namespace JobRunner +{ + /// + /// The class object encompassing the appsettings parameters + /// + [Export] + [Shared] + public class AppConfig + { + [ImportingConstructor] + public AppConfig(IConfiguration configuration) + { + configuration.Bind("JobRunner", this); + } + + public string StorageConnection { get; set; } + public string ServiceBusConnectionString { get; set; } + public string PrimaryQueueName { get; set; } + public string TestQueueName { get; set; } + public string ActiveQueueName { get; set; } + public string ServiceUrl { get; set; } + public string ServiceKeyVaultName { get; set; } + public string AppInsightsIntrumentationKey { get; set; } + public string MicrosoftAuthority { get; set; } + public string ApplicationIdentifierUri { get; set; } + public string ApplicationId { get; set; } + public string SecretKey { get; set; } + public string BlobConnectionString { get; set; } + public string BlobUri { get; set; } + public string EvenHubConnectionString { get; set; } + public string EventHubName { get; set; } + public string IsIotHub { get; set; } + public string Seconds { get; set; } + public string FlowName { get; set; } + public string NormalizationSnippet { get; set; } + public string DatabricksToken { get; set; } + public string SparkType { get; set; } + } +} diff --git a/Services/JobRunner/Connected Services/Application Insights/ConnectedService.json b/Services/JobRunner/Connected Services/Application Insights/ConnectedService.json new file mode 100644 index 000000000..561c8c3f8 --- /dev/null +++ b/Services/JobRunner/Connected Services/Application Insights/ConnectedService.json @@ -0,0 +1,7 @@ +{ + "ProviderId": "Microsoft.ApplicationInsights.ConnectedService.ConnectedServiceProvider", + "Version": "8.14.20131.1", + "GettingStartedDocument": { + "Uri": "https://go.microsoft.com/fwlink/?LinkID=798432" + } +} \ No newline at end of file diff --git a/Services/DataX.Flow/Flow.LiveDataService/Startup.cs b/Services/JobRunner/JobQueueMessage.cs similarity index 53% rename from Services/DataX.Flow/Flow.LiveDataService/Startup.cs rename to Services/JobRunner/JobQueueMessage.cs index 9f1fafb9f..7e2003aac 100644 --- a/Services/DataX.Flow/Flow.LiveDataService/Startup.cs +++ b/Services/JobRunner/JobQueueMessage.cs @@ -1,18 +1,15 @@ -// ********************************************************************* -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License -// ********************************************************************* -namespace Flow.LiveDataService -{ - using Microsoft.AspNetCore.Hosting; - using DataX.ServiceHost.AspNetCore; +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* - public class Startup : StartUpBase +namespace JobRunner +{ + /// + /// Message object to specify a job to be run asynchronously + /// + internal class JobQueueMessage { - public Startup(IHostingEnvironment env) : base(env) - { - } + public string JobKey { get; set; } } } - - diff --git a/Services/JobRunner/JobRunner.cs b/Services/JobRunner/JobRunner.cs new file mode 100644 index 000000000..7d6852d84 --- /dev/null +++ b/Services/JobRunner/JobRunner.cs @@ -0,0 +1,329 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using DataX.Utilities.KeyVault; +using DataX.Utilities.Storage; +using JobRunner.Jobs; +using Microsoft.ApplicationInsights.AspNetCore.Extensions; +using Microsoft.Azure.ServiceBus; +using Microsoft.Extensions.Configuration; +using Microsoft.Extensions.DependencyInjection; +using Microsoft.Extensions.Logging; +using Microsoft.Extensions.Logging.ApplicationInsights; +using Newtonsoft.Json; +using System; +using System.Collections.Generic; +using System.Composition; +using System.Linq; +using System.Reflection; +using System.Runtime.CompilerServices; +using System.Text; +using System.Threading; +using System.Threading.Tasks; + +namespace JobRunner +{ + [Export] + public class JobRunner + { + // We have a distinct primary and test queue to compartmentalize runners. The target queue client for this runner is set in the configuration. + private readonly IQueueClient _primaryQueueClient; + private readonly IQueueClient _testQueueClient; + private const int _Minutes = 20; + + private readonly string _activeQueueName; + private IQueueClient _ActiveQueueClient + { + get + { + if (_activeQueueName == _primaryQueueClient.QueueName) + { + return _primaryQueueClient; + } + return _testQueueClient; + } + } + + private readonly ILogger _logger; + + // We DI our jobs so they can get requirements "for free" + private readonly IServiceCollection _services = new ServiceCollection(); + private readonly IServiceProvider _provider; + + public ScheduledJobTable ScheduledJobTable { get; } + + // Used for mapping serialized job names off of service bus. + private readonly IDictionary _jobKeyTypeMappings = new Dictionary(); + + [ImportingConstructor] + public JobRunner(IConfiguration configuration, ScheduledJobTable jobTable) + { + IServiceCollection services = new ServiceCollection(); + var appConfig = new AppConfig(configuration); + services.AddLogging(loggingBuilder => + { + // Optional: Apply filters to configure LogLevel Trace or above is sent to ApplicationInsights for all + // categories. + loggingBuilder.AddFilter("", LogLevel.Information); + var abc = GetInstrumentationKey(appConfig); + loggingBuilder.AddApplicationInsights(GetInstrumentationKey(appConfig)); + }); + IServiceProvider serviceProvider = services.BuildServiceProvider(); + + _logger = serviceProvider.GetRequiredService>(); + + var sbConnection = KeyVault.GetSecretFromKeyvault(appConfig.ServiceBusConnectionString); + + _primaryQueueClient = new QueueClient(sbConnection, appConfig.PrimaryQueueName); + _testQueueClient = new QueueClient(sbConnection, appConfig.TestQueueName); + _activeQueueName = appConfig.ActiveQueueName; + + // NOTE: If you need to DI other things (e.g. for your jobs) here's where you'd do it, and then just add the injected class into your constructor. + _services.AddSingleton(appConfig) + .AddApplicationInsightsTelemetry(new ApplicationInsightsServiceOptions() + { + EnableAdaptiveSampling = false, + EnableDebugLogger = false, + InstrumentationKey = GetInstrumentationKey(appConfig) + }) + .AddLogging(logging => + { + try + { + // In order to log ILogger logs + logging.AddApplicationInsights(); + // Optional: Apply filters to configure LogLevel Information or above is sent to + // ApplicationInsights for all categories. + logging.AddFilter("", LogLevel.Information); + + // Additional filtering For category starting in "Microsoft", + // only Warning or above will be sent to Application Insights. + logging.AddFilter("Microsoft", LogLevel.Warning); + + } + catch (Exception e) + { + } + }); + _services.AddSingleton(_logger); + + LoadJobTypes(); + _provider = _services.BuildServiceProvider(); + this.ScheduledJobTable = jobTable; + } + private static string GetInstrumentationKey(AppConfig settings) + { + var secretName = settings?.AppInsightsIntrumentationKey; + var vaultName = settings.ServiceKeyVaultName; + + return string.IsNullOrWhiteSpace(secretName) || string.IsNullOrWhiteSpace(vaultName) + ? Guid.Empty.ToString() + : KeyVault.GetSecretFromKeyvault(settings.ServiceKeyVaultName, settings.AppInsightsIntrumentationKey); + } + + /// + /// For a Job type (inheriting from IJob) returns a string name (the class name) for serializing into the job queue + /// + /// The Job Type (inheriting from IJob) to fetch the string name of. + /// A string representing the name of the specified Job type. + private static string GetJobKey() where T : IJob + { + return typeof(T).Name; + } + + /// + /// Created a new instance of the job type specified by the serialized key string. + /// This key string is generated via GetJobKey. + /// + /// String-form name of the Job type for serialization in job queue + /// Instance of the Job type requested. + private IJob CreateJobByKey(string key) + { + if (_jobKeyTypeMappings.TryGetValue(key, out Type job_type)) + { + var job = (IJob)_provider.GetService(job_type); + return job; + } + return null; + } + + /// + /// Enumerates JobRunner.Jobs namespace for all job types and prepares them to be DI initialized. + /// To expose a new Job type, simply add a new Job to the Jobs folder, inheriting from IJob. (See TestJob) + /// + private void LoadJobTypes() { + var assembly = Assembly.GetExecutingAssembly(); + foreach (var type in assembly.GetTypes()) + { + if (IsJobType(type)) + { + _jobKeyTypeMappings.Add(type.Name, type); + _services.AddTransient(type); + } + } + _logger.LogInformation($"LoadedJobRunnerJobs: {_jobKeyTypeMappings.Count}"); + } + + /// + /// Determine if a given Type is a Job (Inherits from IJob, within JobRunner.Jobs namespace) + /// + /// The type to examine + /// A boolean of if this is a job or not. + private bool IsJobType(Type type) + { + var attr = Attribute.GetCustomAttribute(type, typeof(CompilerGeneratedAttribute)); + return string.Equals(type.Namespace, "JobRunner.Jobs", StringComparison.Ordinal) + && !type.IsInterface && attr==null; + } + + /// + /// Insert a specified Job (from JobRunner.Jobs namespace, inheriting from IJob) into the job queue to be run asynchronously. + /// + /// The type of job to be queued. + public async Task EnqueueJobAsync(bool useTestQueue=false) where T : IJob + { + await EnqueueJobByKeyAsync(GetJobKey(), useTestQueue); + } + + private async Task EnqueueJobByKeyAsync(string jobKey, bool useTestQueue=false, string messageId=null) + { + var payload = new JobQueueMessage() { JobKey = jobKey }; + var message = new Message(Encoding.UTF8.GetBytes(JsonConvert.SerializeObject(payload))) + { + // We very intentionally don't want to duplicate-detect, so we add ticks as a nonce. + // For scheduled messages we only send once per interval, we use their timestamp to guarantee sending only once. + MessageId = string.IsNullOrWhiteSpace(messageId) + ? (jobKey + DateTime.UtcNow.Ticks.ToString()) + : (jobKey + messageId) + }; + if (useTestQueue) + { + await _testQueueClient.SendAsync(message); + } + else + { + await _primaryQueueClient.SendAsync(message); + } + } + + /// + /// Callback for a new message appearing on the Queue. Fetches the job specified by the Message payload, + /// and attempts to run it. + /// + /// The Message object to process + /// Cancellation token to halt running + /// async task + private async Task HandleQueuedJobAsync(Message message, CancellationToken token) + { + var body = Encoding.UTF8.GetString(message.Body); + var payload = JsonConvert.DeserializeObject(body); + var job = CreateJobByKey(payload.JobKey); + + _logger.LogInformation("ReceivedJobRunnerJob: " + payload.JobKey.ToString()); + try + { + var jobTask = job.RunAsync(); + // Terminate after 5 minute timeout due to servicebus. + if (await Task.WhenAny(jobTask, Task.Delay(1000 * 60 * 5)) != jobTask) + { + throw new JobRunnerException("JobRunner Job Timeout"); + } + // TODO: Can in the future allow for specifiable job timeout on job baseclass. (and if >5m, immediately completes on queue) + } + catch (Exception e) + { + _logger.LogError(e, $"JobRunnerJobFailure: JobType: {payload.JobKey}; Error: {e.Message}"); + } + + await StorageUtils.RetryOnConflictAsync( + async () => + { + await _ActiveQueueClient.CompleteAsync(message.SystemProperties.LockToken); + }, async () => + { + _logger.LogWarning($"JobRunnerAttemptedToCompleteJob: JobType: {payload.JobKey}"); + await Task.Yield(); + return true; + }); + _logger.LogInformation($"JobRunnerCompletedJob: JobType: {payload.JobKey}"); + } + + private async Task EnqueueScheduledJobsAsync() + { + var scheduledJobs = await this.ScheduledJobTable.RetrieveAllAsync(); + _logger.LogInformation($"JobRunnerFetchedScheduledJobs: {scheduledJobs.Count()}"); + foreach (var schedule in scheduledJobs) + { + if (schedule.ShouldRun) + { + _logger.LogInformation($"JobRunnerEnqueuingScheduledJob: JobKey: {schedule.JobKey} DeduplicationKey: {schedule.DeduplicationId}"); + await EnqueueJobByKeyAsync(schedule.JobKey, schedule.UseTestQueue, schedule.DeduplicationId); + schedule.IncrementLastRunAt(); + await this.ScheduledJobTable.ReplaceAsync(schedule); + } + } + } + + /// + /// Register scheduled jobs here, in absence of a web management experience. + /// + private void RegisterScheduledJobs() + { + this.ScheduledJobTable.CreateOrUpdateScheduledJobAsync( + "TESTSCHEDULEDJOB", + GetJobKey(), + new DateTime(2019, 1, 1, 1, 1, 1), + new TimeSpan(0, 5, 0), + useTestQueue: true).Wait(); + + // run DataX Schema and Query job every few minutes + this.ScheduledJobTable.CreateOrUpdateScheduledJobAsync( + "DataXSchemaAndQueryJob", + GetJobKey(), + new DateTime(2019, 1, 1, 1, 1, 1), + new TimeSpan(0, _Minutes, 0), + useTestQueue: true).Wait(); + + //run DataX mainline job every few minutes. + this.ScheduledJobTable.CreateOrUpdateScheduledJobAsync( + "DataXDeployJob", + GetJobKey(), + new DateTime(2019, 1, 1, 1, 1, 1), + new TimeSpan(0, _Minutes, 0), + useTestQueue: true).Wait(); + } + + /// + /// Primary entry point for JobRunner. Typically only called from the webjob, as it will run forever. + /// + public async Task RunForeverAsync(CancellationToken cancellationToken = new CancellationToken()) + { + RegisterScheduledJobs(); + + var options = new MessageHandlerOptions((e) => + { + _logger.LogError(e.Exception, $"JobRunnerMessageHandlerFailure: Error: {e.Exception.Message}"); + return Task.CompletedTask; + }) + { + MaxConcurrentCalls = 200, + AutoComplete = false + }; + _ActiveQueueClient.RegisterMessageHandler(HandleQueuedJobAsync, options); + + var lastHeartbeat = DateTime.UtcNow; + var heartbeatInterval = new TimeSpan(0, 1, 0); + while (!cancellationToken.IsCancellationRequested) + { + if (DateTime.UtcNow - lastHeartbeat > heartbeatInterval) + { + lastHeartbeat = DateTime.UtcNow; + _logger.LogInformation($"Last Heartbeat: {lastHeartbeat}"); + await EnqueueScheduledJobsAsync(); + } + } + } + } +} diff --git a/Services/JobRunner/JobRunner.csproj b/Services/JobRunner/JobRunner.csproj new file mode 100644 index 000000000..2bfd4198c --- /dev/null +++ b/Services/JobRunner/JobRunner.csproj @@ -0,0 +1,34 @@ + + + + netcoreapp2.2 + InProcess + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/Services/JobRunner/JobRunner.sln b/Services/JobRunner/JobRunner.sln new file mode 100644 index 000000000..4685e7f71 --- /dev/null +++ b/Services/JobRunner/JobRunner.sln @@ -0,0 +1,67 @@ + +Microsoft Visual Studio Solution File, Format Version 12.00 +# Visual Studio 15 +VisualStudioVersion = 15.0.28307.645 +MinimumVisualStudioVersion = 10.0.40219.1 +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "JobRunner", "JobRunner.csproj", "{F4BC35A6-6C4F-45A8-A373-60653E9F4C9E}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "JobRunnerWebJob", "..\JobRunnerWebJob\JobRunnerWebJob.csproj", "{9B1E38B9-DDB4-428F-BBEB-D9F4F0C04C89}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "ScenarioTester", "..\..\Tests\ScenarioTester\ScenarioTester\ScenarioTester.csproj", "{34F0469E-9394-484F-9F60-FBBA71E361C9}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Contract", "..\DataX.Contract\DataX.Contract.csproj", "{424C40A7-FC0D-4B24-888A-A7E0A2D63B98}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.KeyVault", "..\DataX.Utilities\DataX.Utilities.KeyVault\DataX.Utilities.KeyVault.csproj", "{DCCC4084-9B1D-451B-B993-7D2BBCFED2CD}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataXScenarios", "..\..\Tests\DataXScenarios\DataXScenarios\DataXScenarios.csproj", "{FE56E39D-4926-4EC2-90E0-EB7612B657D2}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utility.Blob", "..\DataX.Utilities\DataX.Utility.Blob\DataX.Utility.Blob.csproj", "{CA33FC4C-AE37-450C-81CC-814BA49EAED4}" +EndProject +Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "DataX.Utilities.Composition", "..\DataX.Flow\DataX.Utilities.Composition\DataX.Utilities.Composition.csproj", "{42630955-5D75-48D8-A6CC-1BDB4694F903}" +EndProject +Global + GlobalSection(SolutionConfigurationPlatforms) = preSolution + Debug|Any CPU = Debug|Any CPU + Release|Any CPU = Release|Any CPU + EndGlobalSection + GlobalSection(ProjectConfigurationPlatforms) = postSolution + {F4BC35A6-6C4F-45A8-A373-60653E9F4C9E}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {F4BC35A6-6C4F-45A8-A373-60653E9F4C9E}.Debug|Any CPU.Build.0 = Debug|Any CPU + {F4BC35A6-6C4F-45A8-A373-60653E9F4C9E}.Release|Any CPU.ActiveCfg = Release|Any CPU + {F4BC35A6-6C4F-45A8-A373-60653E9F4C9E}.Release|Any CPU.Build.0 = Release|Any CPU + {9B1E38B9-DDB4-428F-BBEB-D9F4F0C04C89}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {9B1E38B9-DDB4-428F-BBEB-D9F4F0C04C89}.Debug|Any CPU.Build.0 = Debug|Any CPU + {9B1E38B9-DDB4-428F-BBEB-D9F4F0C04C89}.Release|Any CPU.ActiveCfg = Release|Any CPU + {9B1E38B9-DDB4-428F-BBEB-D9F4F0C04C89}.Release|Any CPU.Build.0 = Release|Any CPU + {34F0469E-9394-484F-9F60-FBBA71E361C9}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {34F0469E-9394-484F-9F60-FBBA71E361C9}.Debug|Any CPU.Build.0 = Debug|Any CPU + {34F0469E-9394-484F-9F60-FBBA71E361C9}.Release|Any CPU.ActiveCfg = Release|Any CPU + {34F0469E-9394-484F-9F60-FBBA71E361C9}.Release|Any CPU.Build.0 = Release|Any CPU + {424C40A7-FC0D-4B24-888A-A7E0A2D63B98}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {424C40A7-FC0D-4B24-888A-A7E0A2D63B98}.Debug|Any CPU.Build.0 = Debug|Any CPU + {424C40A7-FC0D-4B24-888A-A7E0A2D63B98}.Release|Any CPU.ActiveCfg = Release|Any CPU + {424C40A7-FC0D-4B24-888A-A7E0A2D63B98}.Release|Any CPU.Build.0 = Release|Any CPU + {DCCC4084-9B1D-451B-B993-7D2BBCFED2CD}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {DCCC4084-9B1D-451B-B993-7D2BBCFED2CD}.Debug|Any CPU.Build.0 = Debug|Any CPU + {DCCC4084-9B1D-451B-B993-7D2BBCFED2CD}.Release|Any CPU.ActiveCfg = Release|Any CPU + {DCCC4084-9B1D-451B-B993-7D2BBCFED2CD}.Release|Any CPU.Build.0 = Release|Any CPU + {FE56E39D-4926-4EC2-90E0-EB7612B657D2}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {FE56E39D-4926-4EC2-90E0-EB7612B657D2}.Debug|Any CPU.Build.0 = Debug|Any CPU + {FE56E39D-4926-4EC2-90E0-EB7612B657D2}.Release|Any CPU.ActiveCfg = Release|Any CPU + {FE56E39D-4926-4EC2-90E0-EB7612B657D2}.Release|Any CPU.Build.0 = Release|Any CPU + {CA33FC4C-AE37-450C-81CC-814BA49EAED4}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {CA33FC4C-AE37-450C-81CC-814BA49EAED4}.Debug|Any CPU.Build.0 = Debug|Any CPU + {CA33FC4C-AE37-450C-81CC-814BA49EAED4}.Release|Any CPU.ActiveCfg = Release|Any CPU + {CA33FC4C-AE37-450C-81CC-814BA49EAED4}.Release|Any CPU.Build.0 = Release|Any CPU + {42630955-5D75-48D8-A6CC-1BDB4694F903}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {42630955-5D75-48D8-A6CC-1BDB4694F903}.Debug|Any CPU.Build.0 = Debug|Any CPU + {42630955-5D75-48D8-A6CC-1BDB4694F903}.Release|Any CPU.ActiveCfg = Release|Any CPU + {42630955-5D75-48D8-A6CC-1BDB4694F903}.Release|Any CPU.Build.0 = Release|Any CPU + EndGlobalSection + GlobalSection(SolutionProperties) = preSolution + HideSolutionNode = FALSE + EndGlobalSection + GlobalSection(ExtensibilityGlobals) = postSolution + SolutionGuid = {BDAF41D8-EBB3-4872-B9B3-048DF71E25F3} + EndGlobalSection +EndGlobal diff --git a/Services/DataX.Flow/Flow.InteractiveQueryService/Startup.cs b/Services/JobRunner/JobRunnerException.cs similarity index 55% rename from Services/DataX.Flow/Flow.InteractiveQueryService/Startup.cs rename to Services/JobRunner/JobRunnerException.cs index f2ff2a1cc..94c19683c 100644 --- a/Services/DataX.Flow/Flow.InteractiveQueryService/Startup.cs +++ b/Services/JobRunner/JobRunnerException.cs @@ -1,18 +1,19 @@ -// ********************************************************************* -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License -// ********************************************************************* -namespace Flow.InteractiveQueryService -{ - using Microsoft.AspNetCore.Hosting; - using DataX.ServiceHost.AspNetCore; +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using System; - public class Startup : StartUpBase +namespace JobRunner +{ + /// + /// Custom Exception for JobRunner + /// + public class JobRunnerException : Exception { - public Startup(IHostingEnvironment env) : base(env) + public JobRunnerException(string message) : base(message) { } } } - - diff --git a/Services/JobRunner/Jobs/DataXDeployJob.cs b/Services/JobRunner/Jobs/DataXDeployJob.cs new file mode 100644 index 000000000..be8003224 --- /dev/null +++ b/Services/JobRunner/Jobs/DataXDeployJob.cs @@ -0,0 +1,123 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using ScenarioTester; +using System; +using DataX.ServerScenarios; +using System.Threading.Tasks; +using Microsoft.Extensions.Logging; +using System.Collections.Generic; +using System.Linq; +using DataX.Utilities.KeyVault; +using DataX.Utility.Blob; + +namespace JobRunner.Jobs +{ + /// + /// Runs through a steel thread scenario for a DataX every few minutes to ensure DataX E2E works for saving and deploying a job. + /// + public class DataXDeployJob : IJob + { + private readonly ScenarioDescription _scenario; + private readonly AppConfig _config; + private readonly ILogger _logger; + private readonly int _scenarioCount = 1; + + public DataXDeployJob(AppConfig config, ILogger logger) + { + _config = config; + _logger = logger; + + _scenario = new ScenarioDescription("DataXMainline", + DataXHost.AcquireToken, + DataXHost.SaveJob, + DataXHost.GenerateConfigs, + DataXHost.RestartJob, + DataXHost.GetFlow + ); + } + /// + /// This is the method that gets called when the job starts running + /// + /// + public async Task RunAsync() + { + if (string.IsNullOrWhiteSpace(_config.ServiceUrl)) + { + string errorMessage = "Server URL is not available."; + _logger.LogError(_scenario.Description, "JobRunner ScenarioTester", new Dictionary() { { "scenario.errorMessage", errorMessage } }); + + throw new InvalidOperationException(errorMessage); + } + + using (var context = new ScenarioContext()) + { + context[Context.ServiceUrl] = _config.ServiceUrl; + context[Context.ApplicationId] = KeyVault.GetSecretFromKeyvault(_config.ApplicationId); + + // The flow config needs to be saved at this location + string blobUri = $"{_config.BlobUri}"; + context[Context.FlowConfigContent] = await Task.Run(() => BlobUtility.GetBlobContent(KeyVault.GetSecretFromKeyvault(_config.BlobConnectionString), blobUri)); + context[Context.ApplicationIdentifierUri] = _config.ApplicationIdentifierUri; + context[Context.SecretKey] = KeyVault.GetSecretFromKeyvault(_config.SecretKey); + context[Context.MicrosoftAuthority] = _config.MicrosoftAuthority; + + using (_logger.BeginScope>>( + new Dictionary { + { "scenario.Description", _scenario.Description }, + { "scenarioCount", _scenarioCount.ToString() }, + { "scenario.Steps", $"[{string.Join(", ", _scenario.Steps.Select(s => s.Method.Name))}]" } + })) + { + // do actual logging inside the scope. All logs inside this will have the properties from the Dictionary used in begin scope. + _logger.LogInformation("JobRunner ScenarioTester: " + _scenario.Description); + + } + + var results = await ScenarioResult.RunAsync(_scenario, context, _scenarioCount); + int iterationCount = 0; + + foreach (var result in results) + { + string scenarioResult = result.Failed ? "failed" : "succeeded"; + + // log failed steps. + foreach (var stepResult in result.StepResults.Where(r => !r.Success)) + { + using (_logger.BeginScope>>( + new Dictionary { + { "Scenario iteration", $"Scenario iteration {_scenario.Description}.{iterationCount} " }, + { "ScenarioResult length", scenarioResult.Length} + })) + { + // do actual logging inside the scope. All logs inside this will have the properties from the Dictionary used in begin scope. + _logger.LogInformation(_scenario.Description); + + } + + if (stepResult.Exception != null) + { + _logger.LogError(stepResult.Exception, _scenario.Description); + } + _logger.LogError(stepResult.Value); + } + + iterationCount++; + } + + //emit metric on how many parallel executions passed. + using (_logger.BeginScope>>( + new Dictionary { + { $"SuccessRate:{_scenario.Description}", $"{(long)((double)results.Count(r => !r.Failed) / _scenarioCount * 100.0)}" } + })) + { + // do actual logging inside the scope. All logs inside this will have the properties from the Dictionary used in begin scope. + _logger.LogInformation(_scenario.Description); + + } + } + } + } +} diff --git a/Services/JobRunner/Jobs/DataXSchemaAndQueryJob.cs b/Services/JobRunner/Jobs/DataXSchemaAndQueryJob.cs new file mode 100644 index 000000000..3fdf73037 --- /dev/null +++ b/Services/JobRunner/Jobs/DataXSchemaAndQueryJob.cs @@ -0,0 +1,138 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using ScenarioTester; +using System; +using DataX.ServerScenarios; +using System.Threading.Tasks; +using Microsoft.Extensions.Logging; +using System.Collections.Generic; +using System.Linq; +using DataX.Utilities.KeyVault; +using DataX.Utility.Blob; +using Newtonsoft.Json; + +namespace JobRunner.Jobs +{ + /// + /// Runs through a steel thread scenario for a DataX every few minutes to ensure DataX E2E works for Interactive query experience, SchemaGen and Live data service. + /// + public class DataXSchemaAndQueryJob : IJob + { + private readonly ScenarioDescription _scenario; + private readonly AppConfig _config; + private readonly ILogger _logger; + private readonly int _scenarioCount = 1; + + public DataXSchemaAndQueryJob(AppConfig config, ILogger logger) + { + _config = config; + _logger = logger; + + _scenario = new ScenarioDescription("DataXSchemaAndQuery", + DataXHost.AcquireToken, + DataXHost.InferSchema, + DataXHost.InitializeKernel, + DataXHost.RefreshSampleAndKernel, + DataXHost.RefreshKernel, + DataXHost.RefreshSample + ); + } + /// + /// This is the method that gets called when the job starts running + /// + /// + public async Task RunAsync() + { + if (string.IsNullOrWhiteSpace(_config.ServiceUrl)) + { + string errorMessage = "Server URL is not available."; + _logger.LogError(_scenario.Description, "JobRunner ScenarioTester", new Dictionary() { { "scenario.errorMessage", errorMessage } }); + + throw new InvalidOperationException(errorMessage); + } + + using (var context = new ScenarioContext()) + { + context[Context.ServiceUrl] = _config.ServiceUrl; + context[Context.ApplicationId] = KeyVault.GetSecretFromKeyvault(_config.ApplicationId); + context[Context.ApplicationIdentifierUri] = _config.ApplicationIdentifierUri; + context[Context.SecretKey] = KeyVault.GetSecretFromKeyvault(_config.SecretKey); + context[Context.MicrosoftAuthority] = _config.MicrosoftAuthority; + context[Context.EventhubConnectionString] = KeyVault.GetSecretFromKeyvault(_config.EvenHubConnectionString); + context[Context.EventHubName] = _config.EventHubName; + context[Context.IsIotHub] = _config.IsIotHub; + context[Context.Seconds] = _config.Seconds; + context[Context.FlowName] = _config.FlowName; + if (_config.SparkType == "databricks") + { + context[Context.DataBricksToken] = KeyVault.GetSecretFromKeyvault(_config.DatabricksToken); + } + else + { + context[Context.DataBricksToken] = ""; + } + context[Context.SparkType] = _config.SparkType; + context[Context.FlowConfigContent] = await Task.Run(() => BlobUtility.GetBlobContent(KeyVault.GetSecretFromKeyvault(_config.BlobConnectionString), _config.BlobUri)); + context[Context.NormalizationSnippet] = JsonConvert.SerializeObject(_config.NormalizationSnippet); + context[Context.KernelId] = ""; + + using (_logger.BeginScope>>( + new Dictionary { + { "scenario.Description", _scenario.Description }, + { "scenarioCount", _scenarioCount.ToString() }, + { "scenario.Steps", $"[{string.Join(", ", _scenario.Steps.Select(s => s.Method.Name))}]" } + })) + { + // do actual logging inside the scope. All logs inside this will have the properties from the Dictionary used in begin scope. + _logger.LogInformation("JobRunner ScenarioTester: " + _scenario.Description); + + } + + var results = await ScenarioResult.RunAsync(_scenario, context, _scenarioCount); + int iterationCount = 0; + + foreach (var result in results) + { + string scenarioResult = result.Failed ? "failed" : "succeeded"; + + // log failed steps. + foreach (var stepResult in result.StepResults.Where(r => !r.Success)) + { + using (_logger.BeginScope>>( + new Dictionary { + { "Scenario iteration", $"Scenario iteration {_scenario.Description}.{iterationCount} " }, + { "ScenarioResult length", scenarioResult.Length} + })) + { + // do actual logging inside the scope. All logs inside this will have the properties from the Dictionary used in begin scope. + _logger.LogInformation(_scenario.Description); + + } + + if (stepResult.Exception != null) + { + _logger.LogError(stepResult.Exception, _scenario.Description); + } + _logger.LogError(stepResult.Value); + } + + iterationCount++; + } + + //emit metric on how many parallel executions passed. + using (_logger.BeginScope>>( + new Dictionary { + { $"SuccessRate:{_scenario.Description}", $"{(long)((double)results.Count(r => !r.Failed) / _scenarioCount * 100.0)}" } + })) + { + // do actual logging inside the scope. All logs inside this will have the properties from the Dictionary used in begin scope. + _logger.LogInformation(_scenario.Description); + + } + } + } + } +} diff --git a/Services/DataX.Flow/Flow.SchemaInferenceService/Startup.cs b/Services/JobRunner/Jobs/IJob.cs similarity index 52% rename from Services/DataX.Flow/Flow.SchemaInferenceService/Startup.cs rename to Services/JobRunner/Jobs/IJob.cs index fafb106fa..a9ebedf19 100644 --- a/Services/DataX.Flow/Flow.SchemaInferenceService/Startup.cs +++ b/Services/JobRunner/Jobs/IJob.cs @@ -1,18 +1,14 @@ -// ********************************************************************* -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License -// ********************************************************************* -namespace Flow.SchemaInferenceService -{ - using Microsoft.AspNetCore.Hosting; - using DataX.ServiceHost.AspNetCore; +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using System.Threading.Tasks; - public class Startup : StartUpBase +namespace JobRunner.Jobs +{ + public interface IJob { - public Startup(IHostingEnvironment env) : base(env) - { - } + Task RunAsync(); } } - - diff --git a/Services/JobRunner/Jobs/TestJob.cs b/Services/JobRunner/Jobs/TestJob.cs new file mode 100644 index 000000000..4833a3409 --- /dev/null +++ b/Services/JobRunner/Jobs/TestJob.cs @@ -0,0 +1,26 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using System; +using System.Threading.Tasks; + +namespace JobRunner.Jobs +{ + /// + /// An example job that shows how a job could be setup + /// + public class TestJob : IJob + { + public TestJob() + { + } + + public async Task RunAsync() + { + Console.WriteLine("TESTJOB"); + await Task.Yield(); + } + } +} diff --git a/Services/JobRunner/MefContainer.cs b/Services/JobRunner/MefContainer.cs new file mode 100644 index 000000000..efcff82de --- /dev/null +++ b/Services/JobRunner/MefContainer.cs @@ -0,0 +1,66 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using DataX.Utilities.Composition; +using Microsoft.Extensions.Configuration; +using System; +using System.Collections.Generic; +using System.Composition.Hosting; +using System.IO; +using System.Reflection; + +namespace JobRunner +{ + public static class MefContainer + { + public static ContainerConfiguration GenerateConfiguration() + { + // Instead of building the CompositionContainer using the ConventionBuilder, we enumerate all the assemblies and create the container with them. + // This will ensure that native dlls are not loaded into container creation which throws exception. + List allAssemblies = new List(); + var executingDir = Path.GetDirectoryName(Assembly.GetExecutingAssembly().Location); + foreach (var file in Directory.EnumerateFiles(executingDir, "*.dll", SearchOption.AllDirectories)) + { + try + { + var assembly = Assembly.LoadFrom(file); + allAssemblies.Add(assembly); + } + catch (BadImageFormatException) + { + // do nothing and skip the assembly to load as it might be the native assemblies + } + } + + var mefConfig = new ContainerConfiguration().WithAssemblies(allAssemblies); + return mefConfig; + } + + public static CompositionHost CreateWithConfiguration(IConfiguration configuration) + { + var mefConfig = GenerateConfiguration().WithProvider(new InstanceExportDescriptorProvider(configuration)); + return mefConfig.CreateContainer(); + } + + public static IConfiguration BuildConfigurationForConsoleApp(string environment = null) + { + if (string.IsNullOrWhiteSpace(environment)) + { + // Load the ability to specify env. in deployed webjob. + environment = Environment.GetEnvironmentVariable("ENVIRONMENT"); + } + + // Load appsettings.json, and any env specific appsettings in the form "apsettings..json" matching the ENVIRONMENT variable. + var config = new ConfigurationBuilder() + .SetBasePath(Directory.GetCurrentDirectory()) + .AddJsonFile("appsettings.json", optional: false, reloadOnChange: true) + .AddJsonFile($"appsettings.{environment}.json", optional: true, reloadOnChange: true) + .AddEnvironmentVariables("DATAX_") + .Build(); + + return config; + } + } +} diff --git a/Services/JobRunner/ObjectModel/ScheduledJobEntity.cs b/Services/JobRunner/ObjectModel/ScheduledJobEntity.cs new file mode 100644 index 000000000..c5c13a52b --- /dev/null +++ b/Services/JobRunner/ObjectModel/ScheduledJobEntity.cs @@ -0,0 +1,122 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using Microsoft.Azure.Cosmos.Table; +using System; +using DataX.Utilities; + +namespace JobRunner +{ + /// + /// Represents a Scheduled Job definition identified by its . + /// + public sealed class ScheduledJobEntity : BaseTableEntity + { + /// + /// The unique name to give a scheduled job. + /// + public string Name + { + get => RowKey; + set => RowKey = value; + } + + /// + /// The type of job to run. + /// + public string JobKey + { + get; + set; + } + + /// + /// The date/time to begin running this scheduled job. + /// + public DateTimeOffset StartRunningAt + { + get; + set; + } + + /// + /// The last date/time this job was run. + /// + public DateTimeOffset LastRunAt + { + get; + set; + } + + /// + /// The interval between scheduled job runs. + /// + [IgnoreProperty] + public TimeSpan Interval + { + get => new TimeSpan(0, 0, IntervalSeconds); + set => IntervalSeconds = (int)value.TotalSeconds; + } + + /// + /// This is the interval in seconds between scheduled job runs. + /// Table storage can't actually handle Timespans, so this is the underlying value for Interval. + /// + public int IntervalSeconds + { + get; + set; + } + + /// + /// Should the job be scheduled on the primary queue or the test queue. Defaults to false. + /// + public bool UseTestQueue + { + get; + set; + } + + + [IgnoreProperty] + public bool ShouldRun => TimeSinceLastRun > Interval; + + [IgnoreProperty] + private DateTimeOffset LatestIntervalStart => LastRunAt == new DateTimeOffset() ? StartRunningAt : LastRunAt; + + [IgnoreProperty] + private TimeSpan TimeSinceLastRun => DateTimeOffset.UtcNow - LatestIntervalStart; + + /// + /// For use when queuing the schedule, to prevent multiple workers double-queuing + /// + [IgnoreProperty] + public string DeduplicationId => (LatestIntervalStart.ToUnixTimeSeconds() + IntervalSeconds).ToString(); + + public void IncrementLastRunAt() + { + // For the purpose of always trying to run at clean intervals of time and not churn on catchup, + // this makes it such that if e.g. 5.5 intervals have passed, we just run once and set the time at 5 intervals passed. + LastRunAt = LatestIntervalStart.AddSeconds(Math.Floor(TimeSinceLastRun.TotalSeconds / Interval.TotalSeconds) * Interval.TotalSeconds); + } + + public ScheduledJobEntity() + { + PartitionKey = SharedPartitionKey; + } + + internal static readonly string SharedPartitionKey = "sharedPartitionKey"; + + public ScheduledJobEntity(string name, string jobKey, DateTimeOffset startAt, TimeSpan interval, bool useTestQueue = false) + { + Name = name; + JobKey = jobKey; + StartRunningAt = startAt; + Interval = interval; + PartitionKey = SharedPartitionKey; + UseTestQueue = useTestQueue; + } + } +} diff --git a/Services/JobRunner/ObjectModel/ScheduledJobTable.cs b/Services/JobRunner/ObjectModel/ScheduledJobTable.cs new file mode 100644 index 000000000..d1995d5ef --- /dev/null +++ b/Services/JobRunner/ObjectModel/ScheduledJobTable.cs @@ -0,0 +1,52 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using DataX.Contract; +using DataX.Utilities.Storage; +using System; +using System.Composition; +using System.Threading.Tasks; + +namespace JobRunner +{ + /// + /// CosmosTable will be storing the scheduled jobs information like the start running at time. + /// + [Export] + [Shared] + public class ScheduledJobTable : StorageTableBase + { + [ImportingConstructor] + public ScheduledJobTable(TableFactory tableFactory) : + base(tableFactory.GetOrCreateTable("scheduledJobs")) + { + } + + public Task RetrieveAsync(string name) + { + return base.RetrieveAsync(ScheduledJobEntity.SharedPartitionKey, name); + } + + public async Task CreateOrUpdateScheduledJobAsync(string name, string jobKey, DateTimeOffset startAt, TimeSpan interval, bool useTestQueue = false) + { + Ensure.Equals(string.IsNullOrWhiteSpace(jobKey), false); + Ensure.Equals(string.IsNullOrWhiteSpace(name), false); + var newSchedule = new ScheduledJobEntity(name, jobKey, startAt, interval, useTestQueue); + + return await StorageUtils.RetryOnConflictAsync( + async () => + { + try + { + var existingSchedule = await this.RetrieveAsync(name); + newSchedule.LastRunAt = existingSchedule.LastRunAt; + } + catch { } + return await this.Table.InsertOrReplace(newSchedule).ContinueOnAnyContext(); + } + ); + } + } +} diff --git a/Services/JobRunner/Program.cs b/Services/JobRunner/Program.cs new file mode 100644 index 000000000..b41ad68d9 --- /dev/null +++ b/Services/JobRunner/Program.cs @@ -0,0 +1,25 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using Microsoft.AspNetCore.Hosting; +using Microsoft.Extensions.DependencyInjection; +using Microsoft.Extensions.Logging; + +namespace JobRunner +{ + /// + /// JobRunnerWebJob is the entrypoint of the continuously running functionality of JobRunner. + /// All logic, libs, etc, are in the core JobRunner project. This is only a wrapper. + /// + internal class Program + { + public static void Main(string[] args) + { + var host = StartupHelper.HostWebWith(args); + var logger = host.Services.GetRequiredService>(); + host.Run(); + } + } +} diff --git a/Services/JobRunner/Properties/launchSettings.json b/Services/JobRunner/Properties/launchSettings.json new file mode 100644 index 000000000..18dad1298 --- /dev/null +++ b/Services/JobRunner/Properties/launchSettings.json @@ -0,0 +1,27 @@ +{ + "iisSettings": { + "windowsAuthentication": false, + "anonymousAuthentication": true, + "iisExpress": { + "applicationUrl": "http://localhost:19117", + "sslPort": 44362 + } + }, + "profiles": { + "IIS Express": { + "commandName": "IISExpress", + "launchBrowser": true, + "environmentVariables": { + "ASPNETCORE_ENVIRONMENT": "Development" + } + }, + "JobRunner": { + "commandName": "Project", + "launchBrowser": true, + "applicationUrl": "https://localhost:5001;http://localhost:5000", + "environmentVariables": { + "ASPNETCORE_ENVIRONMENT": "Development" + } + } + } +} \ No newline at end of file diff --git a/Services/JobRunner/Startup.cs b/Services/JobRunner/Startup.cs new file mode 100644 index 000000000..84fb787b3 --- /dev/null +++ b/Services/JobRunner/Startup.cs @@ -0,0 +1,99 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using System; +using Microsoft.AspNetCore.Builder; +using Microsoft.AspNetCore.Hosting; +using Microsoft.AspNetCore.Http; +using Microsoft.Extensions.Configuration; +using Microsoft.Extensions.DependencyInjection; +using Microsoft.ApplicationInsights.AspNetCore.Extensions; +using DataX.Utilities.KeyVault; +using Microsoft.Extensions.Logging; +using Microsoft.Extensions.Logging.ApplicationInsights; + +namespace JobRunner +{ + internal class Startup + { + public IConfiguration Configuration { get; } + + public Startup(IConfiguration configuration) + { + this.Configuration = configuration; + } + + // This method gets called by the runtime. Use this method to add services to the container. + // For more information on how to configure your application, visit https://go.microsoft.com/fwlink/?LinkID=398940 + public void ConfigureServices(IServiceCollection services, AppConfig settings) + { + services + .AddApplicationInsightsTelemetry(new ApplicationInsightsServiceOptions() + { + EnableAdaptiveSampling = false, + EnableDebugLogger = false, + InstrumentationKey = GetInstrumentationKey(settings) + }) + .AddLogging(logging => + { + try + { + // In order to log ILogger logs + logging.AddApplicationInsights(); + // Optional: Apply filters to configure LogLevel Information or above is sent to + // ApplicationInsights for all categories. + logging.AddFilter("", LogLevel.Information); + + // Additional filtering For category starting in "Microsoft", + // only Warning or above will be sent to Application Insights. + logging.AddFilter("Microsoft", LogLevel.Warning); + + } + catch (Exception e) + { + } + }); + + } + + + public void Configure(IApplicationBuilder app) + { + var hostingEnvironment = app.ApplicationServices.GetService(); + var loggerFactory = app.ApplicationServices.GetService(); + + Configure(app, hostingEnvironment, loggerFactory); + } + + // This method gets called by the runtime. Use this method to configure the HTTP request pipeline. + public void Configure(IApplicationBuilder app, IHostingEnvironment env, ILoggerFactory loggerFactory) + { + if (env.IsDevelopment()) + { + app.UseDeveloperExceptionPage(); + } + + app.Run(async (context) => + { + await context.Response.WriteAsync("Hello World!"); + }); + } + + /// + /// Helper method for getting the ApplicationInsights key from the keyvault + /// + /// This is an object that contains the appsettings values as set in the appsettings.json + /// + private static string GetInstrumentationKey(AppConfig settings) + { + var secretName = settings?.AppInsightsIntrumentationKey; + var vaultName = settings?.ServiceKeyVaultName; + + return string.IsNullOrWhiteSpace(secretName) || string.IsNullOrWhiteSpace(vaultName) + ? Guid.Empty.ToString() + : KeyVault.GetSecretFromKeyvault(settings.ServiceKeyVaultName, settings.AppInsightsIntrumentationKey); + } + } +} diff --git a/Services/JobRunner/StartupHelper.cs b/Services/JobRunner/StartupHelper.cs new file mode 100644 index 000000000..253cf287a --- /dev/null +++ b/Services/JobRunner/StartupHelper.cs @@ -0,0 +1,32 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using Microsoft.AspNetCore; +using Microsoft.AspNetCore.Hosting; +using Microsoft.Extensions.Configuration; + +namespace JobRunner +{ + /// + /// Helper that gets called from startup for the JobRunner service. + /// + public static class StartupHelper + { + public static IWebHost HostWebWith(string[] args) where TStartup : class + { + var host = CreateWebHostBuilder(args).Build(); + return host; + } + + public static IWebHostBuilder CreateWebHostBuilder(string[] args) where TStartup : class + { + return WebHost.CreateDefaultBuilder(args) + .ConfigureAppConfiguration((hostingContext, config) => config.AddCommandLine(args).AddEnvironmentVariables(prefix: "DATAX_")) + .UseStartup(); + + } + + } +} diff --git a/Services/JobRunner/TableFactory.cs b/Services/JobRunner/TableFactory.cs new file mode 100644 index 000000000..fa8566619 --- /dev/null +++ b/Services/JobRunner/TableFactory.cs @@ -0,0 +1,32 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Microsoft.Azure.Cosmos.Table; +using DataX.Utilities.KeyVault; +using System.Composition; + +namespace JobRunner +{ + [Export] + [Shared] + public class TableFactory + { + private readonly CloudTableClient _tableClient; + + [ImportingConstructor] + public TableFactory(AppConfig config) + { + var storageConnection = KeyVault.GetSecretFromKeyvault(config.StorageConnection); + var storageAccount = CloudStorageAccount.Parse(storageConnection); + _tableClient = storageAccount.CreateCloudTableClient(); + } + + public CloudTable GetOrCreateTable(string tableName) + { + var table = _tableClient.GetTableReference(tableName); + table.CreateIfNotExists(); + return table; + } + } +} diff --git a/Services/JobRunner/appsettings.json b/Services/JobRunner/appsettings.json new file mode 100644 index 000000000..def9159a7 --- /dev/null +++ b/Services/JobRunner/appsettings.json @@ -0,0 +1,8 @@ +{ + "Logging": { + "LogLevel": { + "Default": "Warning" + } + }, + "AllowedHosts": "*" +} diff --git a/Services/JobRunnerWebJob/JobRunnerWebJob.csproj b/Services/JobRunnerWebJob/JobRunnerWebJob.csproj new file mode 100644 index 000000000..f74e5d779 --- /dev/null +++ b/Services/JobRunnerWebJob/JobRunnerWebJob.csproj @@ -0,0 +1,52 @@ + + + Exe + netcoreapp2.2 + + + + + + + + + + + + + + + System + + + System.Data + + + System.Xml + + + + + PreserveNewest + PreserveNewest + + + PreserveNewest + + + Always + + + PreserveNewest + + + Always + + + Always + + + + + + \ No newline at end of file diff --git a/Services/JobRunnerWebJob/Program.cs b/Services/JobRunnerWebJob/Program.cs new file mode 100644 index 000000000..2a0968ca6 --- /dev/null +++ b/Services/JobRunnerWebJob/Program.cs @@ -0,0 +1,17 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +namespace JobRunner +{ + public class Program + { + public static void Main(string[] args) + { + var config = MefContainer.BuildConfigurationForConsoleApp(); + var mef = MefContainer.CreateWithConfiguration(config); + mef.GetExport().RunForeverAsync().Wait(); + } + } +} diff --git a/Services/JobRunnerWebJob/Properties/launchSettings.json b/Services/JobRunnerWebJob/Properties/launchSettings.json new file mode 100644 index 000000000..d69f22ab0 --- /dev/null +++ b/Services/JobRunnerWebJob/Properties/launchSettings.json @@ -0,0 +1,10 @@ +{ + "profiles": { + "JobRunnerWebJob": { + "commandName": "Project", + "environmentVariables": { + "ENVIRONMENT": "Development" + } + } + } +} \ No newline at end of file diff --git a/Services/JobRunnerWebJob/Readme.md b/Services/JobRunnerWebJob/Readme.md new file mode 100644 index 000000000..40739efdc --- /dev/null +++ b/Services/JobRunnerWebJob/Readme.md @@ -0,0 +1,8 @@ +How to setup your own Job: + +1. Create a Job class inheriting from `IJob` under the JobRunner project. +2. You need to schedule the job under `RegisterScheduledJobs` in the JobRunner project. + +Debug locally +1. Update the ActiveQueue in the appsettings to ensure that the correct queue is accessed in the debugging. + diff --git a/Services/JobRunnerWebJob/Settings.job b/Services/JobRunnerWebJob/Settings.job new file mode 100644 index 000000000..81861f1d4 --- /dev/null +++ b/Services/JobRunnerWebJob/Settings.job @@ -0,0 +1,22 @@ +{ + + // Examples: + + // Runs every minute + // "schedule": "0 * * * * *" + + // Runs every 15 minutes + // "schedule": "0 */15 * * * *" + + // Runs every hour (i.e. whenever the count of minutes is 0) + // "schedule": "0 0 * * * *" + + // Runs every hour from 9 AM to 5 PM + // "schedule": "0 0 9-17 * * *" + + // Runs at 9:30 AM every day + // "schedule": "0 30 9 * * *" + + // Runs at 9:30 AM every week day + // "schedule": "0 30 9 * * 1-5" +} \ No newline at end of file diff --git a/Services/JobRunnerWebJob/appsettings.Development.json b/Services/JobRunnerWebJob/appsettings.Development.json new file mode 100644 index 000000000..f9a5e7888 --- /dev/null +++ b/Services/JobRunnerWebJob/appsettings.Development.json @@ -0,0 +1,26 @@ +{ + "JobRunner": { + "StorageConnection": "", + "ServiceBusConnectionString": "", + "PrimaryQueueName": "", + "TestQueueName": "", + "ActiveQueueName": "", + "ServiceUrl": "", + "ServiceKeyVaultName": "", + "AppInsightsIntrumentationKey": "", + "MicrosoftAuthority": "", + "ApplicationIdentifierUri": "", + "ApplicationId": "", + "SecretKey": "", + "BlobConnectionString": "", + "BlobUri": "", + "EvenHubConnectionString": "", + "EventHubName": "", + "IsIotHub": "", + "Seconds": "", + "FlowName": "", + "NormalizationSnippet": "", + "DatabricksToken": "", + "SparkType": "" + } +} \ No newline at end of file diff --git a/Services/JobRunnerWebJob/appsettings.json b/Services/JobRunnerWebJob/appsettings.json new file mode 100644 index 000000000..5e163ea85 --- /dev/null +++ b/Services/JobRunnerWebJob/appsettings.json @@ -0,0 +1,26 @@ +{ + "JobRunner": { + "StorageConnection": "", + "ServiceBusConnectionString": "", + "PrimaryQueueName": "", + "TestQueueName": "", + "ActiveQueueName": "", + "ServiceUrl": "", + "ServiceKeyVaultName": "", + "AppInsightsIntrumentation": "", + "MicrosoftAuthority": "", + "ApplicationIdentifierUri": "", + "ApplicationId": "", + "SecretKey": "", + "BlobConnectionString": "", + "BlobUri": "", + "EvenHubConnectionString": "", + "EventHubName": "", + "IsIotHub": "", + "Seconds": "", + "FlowName": "", + "NormalizationSnippet": "", + "DatabricksToken": "", + "SparkType": "" + } +} \ No newline at end of file diff --git a/Services/JobRunnerWebJob/run.cmd b/Services/JobRunnerWebJob/run.cmd new file mode 100644 index 000000000..d9af12240 --- /dev/null +++ b/Services/JobRunnerWebJob/run.cmd @@ -0,0 +1 @@ +dotnet JobRunnerWebJob.dll \ No newline at end of file diff --git a/Services/codecoverage.runsettings b/Services/codecoverage.runsettings new file mode 100644 index 000000000..e2f9fa69b --- /dev/null +++ b/Services/codecoverage.runsettings @@ -0,0 +1,25 @@ + + + + + + + + + + .*\DataX.*.dll + + + .*tests.dll + .*test.dll + Newtonsoft.Json.dll + + + + + + + + \ No newline at end of file diff --git a/Tests/.gitignore b/Tests/.gitignore new file mode 100644 index 000000000..bcfc4e173 --- /dev/null +++ b/Tests/.gitignore @@ -0,0 +1,91 @@ +# User-specific files +*.suo +*.user +*.userosscache +*.sln.docstates + +*.bak + +# User-specific files (MonoDevelop/Xamarin Studio) +*.userprefs + +# Build results +[Dd]ebug/ +[Dd]ebugPublic/ +[Rr]elease/ +[Rr]eleases/ +[Xx]64/ +[Xx]86/ +[Bb]uild/ +bld/ +[Bb]in/ +[Oo]bj/ + +# Visual Studio 2015 cache/options directory +.vs/ +# Uncomment if you have tasks that create the project's static files in wwwroot +#wwwroot/ + +slnx.sqlite + +# MSTest test Results +[Tt]est[Rr]esult*/ +[Bb]uild[Ll]og.* + +# NUNIT +*.VisualState.xml +TestResult.xml + +# Build Results of an ATL Project +[Dd]ebugPS/ +[Rr]eleasePS/ +dlldata.c + +# DNX +project.lock.json +artifacts/ + +*_i.c +*_p.c +*_i.h +*.ilk +*.meta +*.obj +*.pch +*.pdb +*.pgc +*.pgd +*.rsp +*.sbr +*.tlb +*.tli +*.tlh +*.tmp +*.tmp_proj +*.log +*.vspscc +*.vssscc +.builds +*.pidb +*.svclog +*.scc + +# Chutzpah Test files +_Chutzpah* + +# Visual C++ cache files +ipch/ +*.aps +*.ncb +*.opendb +*.opensdf +*.sdf +*.cachefile +*.VC.db + +# Visual Studio profiler +*.psess +*.vsp +*.vspx +*.sap + diff --git a/Tests/DataXScenarios/DataXScenarios.sln b/Tests/DataXScenarios/DataXScenarios.sln new file mode 100644 index 000000000..de896444e --- /dev/null +++ b/Tests/DataXScenarios/DataXScenarios.sln @@ -0,0 +1,25 @@ + +Microsoft Visual Studio Solution File, Format Version 12.00 +# Visual Studio 15 +VisualStudioVersion = 15.0.28307.645 +MinimumVisualStudioVersion = 10.0.40219.1 +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "DataXScenarios", "DataXScenarios\DataXScenarios.csproj", "{8E4E7FA5-01B7-49FB-83D7-2105EE975603}" +EndProject +Global + GlobalSection(SolutionConfigurationPlatforms) = preSolution + Debug|Any CPU = Debug|Any CPU + Release|Any CPU = Release|Any CPU + EndGlobalSection + GlobalSection(ProjectConfigurationPlatforms) = postSolution + {8E4E7FA5-01B7-49FB-83D7-2105EE975603}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {8E4E7FA5-01B7-49FB-83D7-2105EE975603}.Debug|Any CPU.Build.0 = Debug|Any CPU + {8E4E7FA5-01B7-49FB-83D7-2105EE975603}.Release|Any CPU.ActiveCfg = Release|Any CPU + {8E4E7FA5-01B7-49FB-83D7-2105EE975603}.Release|Any CPU.Build.0 = Release|Any CPU + EndGlobalSection + GlobalSection(SolutionProperties) = preSolution + HideSolutionNode = FALSE + EndGlobalSection + GlobalSection(ExtensibilityGlobals) = postSolution + SolutionGuid = {79C8A6A1-87CF-4DC6-8052-E7571AECA2B1} + EndGlobalSection +EndGlobal diff --git a/Tests/DataXScenarios/DataXScenarios/Context.cs b/Tests/DataXScenarios/DataXScenarios/Context.cs new file mode 100644 index 000000000..f19f25ec6 --- /dev/null +++ b/Tests/DataXScenarios/DataXScenarios/Context.cs @@ -0,0 +1,41 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +namespace DataX.ServerScenarios +{ + /// + /// Shared context for all the scenario steps. Used to pass in parameters + /// as input and output for steps. + /// + public class Context + { + public const string ServiceUrl = "server"; + public const string AuthToken = "authToken"; + public const string MicrosoftAuthority = "microsoftAuthority"; + public const string ApplicationIdentifierUri = "applicationIdentifierUri"; + public const string ApplicationId = "applicationId"; + public const string ApplicationName = "applicationName"; + public const string SecretKey = "secretKey"; + public const string AccessTokenType = "accessTokenType"; + public const string FlowName = "flowName"; + public const string GenerateConfigsRuntimeConfigFolder = "generateConfigsRuntimeConfigFolder"; + public const string AuthResult = "authResult"; + public const string RestartJobsName = "restartJobsName"; + public const string FlowConfig = "flowConfig"; + public const string FlowConfigContent = "flowConfigContent"; + public const string InputSchema = "InputSchema"; + public const string EventhubConnectionString = "eventhubConnectionString"; + public const string EventHubName = "eventHubName"; + public const string IsIotHub = "isIotHub"; + public const string InferSchemaInputJson = "inferSchemaInputJson"; + public const string Seconds = "seconds"; + public const string KernelId = "kernelId"; + public const string InitializeKernelJson = "initializeKernelJson"; + public const string NormalizationSnippet = "normalizationSnippet"; + public const string DataBricksToken = "dataBricksToken"; + public const string SparkType = "sparkType"; + public const string StartJobName = "startJobName"; + } +} \ No newline at end of file diff --git a/Tests/DataXScenarios/DataXScenarios/DataXScenarios.csproj b/Tests/DataXScenarios/DataXScenarios/DataXScenarios.csproj new file mode 100644 index 000000000..3324f7cc4 --- /dev/null +++ b/Tests/DataXScenarios/DataXScenarios/DataXScenarios.csproj @@ -0,0 +1,15 @@ + + + + netcoreapp2.2 + + + + + + + + + + + diff --git a/Tests/DataXScenarios/DataXScenarios/Helper.cs b/Tests/DataXScenarios/DataXScenarios/Helper.cs new file mode 100644 index 000000000..551b688a6 --- /dev/null +++ b/Tests/DataXScenarios/DataXScenarios/Helper.cs @@ -0,0 +1,25 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using DataX.ServerScenarios; +using ScenarioTester; + +namespace DataXScenarios +{ + /// + /// Helper class for the scnearios + /// + public class Helper + { + /// + /// Creating a helper function for constructing the InitializeKernelJson + /// + /// ScenarioContext + /// + public string GetInitializeKernelJson(ScenarioContext context) + { + return $"{{\"name\": \"{context[Context.FlowName] as string}\", \"userName\": \"{context[Context.FlowName] as string}\", \"eventhubConnectionString\": \"{context[Context.EventhubConnectionString] as string}\", \"eventHubNames\": \"{context[Context.EventHubName] as string}\", \"inputType\": \"iothub\", \"inputSchema\": {context[Context.InputSchema] as string}, \"kernelId\": \"{context[Context.KernelId] as string}\", \"normalizationSnippet\": {context[Context.NormalizationSnippet] as string}, \"databricksToken\": \"{context[Context.DataBricksToken] as string}\"}}"; + } + } +} diff --git a/Tests/DataXScenarios/DataXScenarios/InteractiveQueryAndSchemaGenScenarios.cs b/Tests/DataXScenarios/DataXScenarios/InteractiveQueryAndSchemaGenScenarios.cs new file mode 100644 index 000000000..926aa4f24 --- /dev/null +++ b/Tests/DataXScenarios/DataXScenarios/InteractiveQueryAndSchemaGenScenarios.cs @@ -0,0 +1,124 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using ScenarioTester; +using Newtonsoft.Json; +using Newtonsoft.Json.Linq; +using System; +using System.Text; +using System.Threading.Tasks; +using DataXScenarios; + +namespace DataX.ServerScenarios +{ + /// + /// Partial class defined such that the steps can be defined for the job + /// + public partial class DataXHost + { + [Step("inferSchema")] + public static StepResult InferSchema(ScenarioContext context) + { + var baseAddress = $"{context[Context.ServiceUrl] as string}/api/DataX.Flow/Flow.SchemaInferenceService/inputdata/inferschema"; + context[Context.InferSchemaInputJson] = $"{{\"name\": \"{context[Context.FlowName] as string}\", \"userName\": \"{context[Context.FlowName] as string}\", \"eventhubConnectionString\": \"{context[Context.EventhubConnectionString] as string}\", \"eventHubNames\": \"{context[Context.EventHubName] as string}\", \"inputType\": \"iothub\", \"seconds\": \"{context[Context.Seconds] as string}\", \"databricksToken\": \"{context[Context.DataBricksToken] as string}\"}}"; + string jsonResult = Request.Post(baseAddress, + RequestContent.EncodeAsJson( + JObject.Parse(context[Context.InferSchemaInputJson] as string)), + context[Context.AuthToken] as string); + dynamic result = JObject.Parse(jsonResult); + context[Context.InputSchema] = JsonConvert.SerializeObject((string)result.result.Schema); + return new StepResult(!string.IsNullOrWhiteSpace(context[Context.InputSchema] as string), + nameof(InferSchema), + $"Inferring Schema '{context[Context.InputSchema]}' "); + } + + [Step("initializeKernel")] + public static StepResult InitializeKernel(ScenarioContext context) + { + var baseAddress = $"{context[Context.ServiceUrl] as string}/api/DataX.Flow/Flow.InteractiveQueryService/kernel"; + + Helper helper = new Helper(); + context[Context.InitializeKernelJson] = helper.GetInitializeKernelJson(context); + string jsonResult = Request.Post(baseAddress, + RequestContent.EncodeAsJson( + JObject.Parse(context[Context.InitializeKernelJson] as string)), + context[Context.AuthToken] as string); + + dynamic result = JObject.Parse(jsonResult); + context[Context.KernelId] = (string)result.result.result; + + context[Context.InitializeKernelJson] = helper.GetInitializeKernelJson(context); + + + return new StepResult(!(string.IsNullOrWhiteSpace(context[Context.KernelId] as string) && (string) result.result.message==""), + nameof(InitializeKernel), + $"Initialize a kernel '{context[Context.KernelId]}' "); + } + + [Step("refreshKernel")] + public static StepResult RefreshKernel(ScenarioContext context) + { + var baseAddress = $"{context[Context.ServiceUrl] as string}/api/DataX.Flow/Flow.InteractiveQueryService/kernel/refresh"; + + string jsonResult = Request.Post(baseAddress, + RequestContent.EncodeAsJson( + JObject.Parse(context[Context.InitializeKernelJson] as string)), + context[Context.AuthToken] as string); + + dynamic result = JObject.Parse(jsonResult); + context[Context.KernelId] = (string)result.result.result; + return new StepResult(!(string.IsNullOrWhiteSpace(context[Context.KernelId] as string) && (string)result.result.message == ""), + nameof(RefreshKernel), + $"Refresh the kernel '{context[Context.KernelId]}' "); + } + + [Step("refreshSample")] + public static StepResult RefreshSample(ScenarioContext context) + { + var baseAddress = $"{context[Context.ServiceUrl] as string}/api/DataX.Flow/Flow.SchemaInferenceService/inputdata/refreshsample"; + + string jsonResult = Request.Post(baseAddress, + RequestContent.EncodeAsJson( + JObject.Parse(context[Context.InferSchemaInputJson] as string)), + context[Context.AuthToken] as string); + dynamic result = JObject.Parse(jsonResult); + return new StepResult(((string)result.result).Contains("success"), + nameof(RefreshSample), + $"Refreshing Sample"); + } + + [Step("refreshSampleAndKernel")] + public static StepResult RefreshSampleAndKernel(ScenarioContext context) + { + var baseAddress = $"{context[Context.ServiceUrl] as string}/api/DataX.Flow/Flow.LiveDataService/inputdata/refreshsampleandkernel"; + context[Context.InitializeKernelJson] = new Helper().GetInitializeKernelJson(context); + + string jsonResult = Request.Post(baseAddress, + RequestContent.EncodeAsJson( + JObject.Parse(context[Context.InitializeKernelJson] as string)), + context[Context.AuthToken] as string); + + dynamic result = JObject.Parse(jsonResult); + context[Context.KernelId] = (string)result.result.result; + return new StepResult(!(string.IsNullOrWhiteSpace(context[Context.KernelId] as string) && (string)result.result.message == ""), + nameof(RefreshSampleAndKernel), + $"Refresh the sample and kernel '{context[Context.KernelId]}' "); + } + + [Step("deleteKernel")] + public static StepResult DeleteKernel(ScenarioContext context) + { + var baseAddress = $"{context[Context.ServiceUrl] as string}/api/DataX.Flow/Flow.InteractiveQueryService/kernel/delete"; + string json = JsonConvert.SerializeObject((string)context[Context.KernelId]); + string jsonResult = Request.Post(baseAddress, + new RequestContent(Encoding.UTF8.GetBytes(json), "application/json"), + context[Context.AuthToken] as string); + dynamic result = JObject.Parse(jsonResult); + return new StepResult(((string)result.result).Contains("Success"), + nameof(DeleteKernel), + $"Delete the kernel '{context[Context.KernelId]}' "); + } + + } +} \ No newline at end of file diff --git a/Tests/DataXScenarios/DataXScenarios/SaveAndDeploy.cs b/Tests/DataXScenarios/DataXScenarios/SaveAndDeploy.cs new file mode 100644 index 000000000..e06849bd1 --- /dev/null +++ b/Tests/DataXScenarios/DataXScenarios/SaveAndDeploy.cs @@ -0,0 +1,133 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using Microsoft.Azure.Services.AppAuthentication; +using Microsoft.IdentityModel.Clients.ActiveDirectory; +using ScenarioTester; +using Newtonsoft.Json; +using Newtonsoft.Json.Linq; +using System; +using System.Text; +using System.Threading.Tasks; + +namespace DataX.ServerScenarios +{ + /// + /// Partial class defined such that the steps can be defined for the job + /// + public partial class DataXHost + { + + [Step("acquireToken")] + public static StepResult AcquireToken(ScenarioContext context) + { + var tokenProvider = new AzureServiceTokenProvider(); + GetS2SAccessTokenForProdMSAAsync(context).Wait(); + + return new StepResult(!string.IsNullOrWhiteSpace(context[Context.AuthToken] as string), + nameof(AcquireToken), "acquired a bearer token"); + } + static public async Task GetS2SAccessTokenForProdMSAAsync(ScenarioContext context) + { + await GetS2SAccessToken(context, context[Context.MicrosoftAuthority] as string, context[Context.ApplicationIdentifierUri] as string, context[Context.ApplicationId] as string, context[Context.SecretKey] as string); + } + + static async Task GetS2SAccessToken(ScenarioContext context, string authority, string resource, string clientId, string clientSecret) + { + var clientCredential = new ClientCredential(clientId, clientSecret); + AuthenticationContext authContext = new AuthenticationContext(authority, false); + AuthenticationResult authenticationResult = await authContext.AcquireTokenAsync( + resource, // the resource (app) we are going to access with the token + clientCredential); // the client credentials + context[Context.AuthToken] = authenticationResult.AccessToken; + context[Context.AuthResult] = authenticationResult; + context[Context.AccessTokenType] = authenticationResult.AccessTokenType; + } + + [Step("saveJob")] + public static StepResult SaveJob(ScenarioContext context) + { + var baseAddress = $"{context[Context.ServiceUrl] as string}/api/DataX.Flow/Flow.ManagementService/flow/save"; + string jsonResult = Request.Post(baseAddress, + RequestContent.EncodeAsJson( + JObject.Parse(context[Context.FlowConfigContent] as string)), + context[Context.AuthToken] as string); + + dynamic result = JObject.Parse(jsonResult); + context[Context.FlowName] = (string)result.result.name; + return new StepResult(!string.IsNullOrWhiteSpace(context[Context.FlowName] as string), + nameof(SaveJob), + $"created a flow '{context[Context.FlowName]}' "); + } + + [Step("startJob")] + public static StepResult StartJob(ScenarioContext context) + { + var baseAddress = $"{context[Context.ServiceUrl] as string}/api/DataX.Flow/Flow.ManagementService/flow/startjobs"; + string json = JsonConvert.SerializeObject((string)context[Context.FlowName]); + string jsonResult = Request.Post(baseAddress, + new RequestContent(Encoding.UTF8.GetBytes(json), "application/json"), + context[Context.AuthToken] as string); + dynamic result = JObject.Parse(jsonResult); + context[Context.RestartJobsName] = (string)result.result.IsSuccess; + string startJobName = context[Context.StartJobName] as string; + + return new StepResult(!string.IsNullOrWhiteSpace(startJobName), + nameof(StartJob), + $"created configs for the flow: '{startJobName}' "); + } + + [Step("generateConfigs")] + public static StepResult GenerateConfigs(ScenarioContext context) + { + var baseAddress = $"{context[Context.ServiceUrl] as string}/api/DataX.Flow/Flow.ManagementService/flow/generateconfigs"; + string json = JsonConvert.SerializeObject((string)context[Context.FlowName]); + string jsonResult = Request.Post(baseAddress, + new RequestContent(Encoding.UTF8.GetBytes(json), "application/json"), + context[Context.AuthToken] as string); + + dynamic result = JObject.Parse(jsonResult); + context[Context.GenerateConfigsRuntimeConfigFolder] = (string)result.result.Properties.runtimeConfigFolder; + + string generateConfigsRuntimeConfigFolder = context[Context.GenerateConfigsRuntimeConfigFolder] as string; + + return new StepResult(!string.IsNullOrWhiteSpace(generateConfigsRuntimeConfigFolder), + nameof(GenerateConfigs), + $"created configs for the flow: '{generateConfigsRuntimeConfigFolder}' "); + } + + [Step("restartJob")] + public static StepResult RestartJob(ScenarioContext context) + { + var baseAddress = $"{context[Context.ServiceUrl] as string}/api/DataX.Flow/Flow.ManagementService/flow/restartjobs"; + string json = JsonConvert.SerializeObject((string)context[Context.FlowName]); + string jsonResult = Request.Post(baseAddress, + new RequestContent(Encoding.UTF8.GetBytes(json), "application/json"), + context[Context.AuthToken] as string); + dynamic result = JObject.Parse(jsonResult); + context[Context.RestartJobsName] = (string)result.result.IsSuccess; + string restartJobsName = context[Context.RestartJobsName] as string; + + return new StepResult(!string.IsNullOrWhiteSpace(restartJobsName), + nameof(RestartJob), + $"created configs for the flow: '{restartJobsName}' "); + } + + [Step("getFlow")] + public static StepResult GetFlow(ScenarioContext context) + { + var baseAddress = $"{context[Context.ServiceUrl] as string}/api/DataX.Flow/Flow.ManagementService/flow/get?flowName={context[Context.FlowName] as string}"; + + string jsonResult = Request.Get(baseAddress, + context[Context.AuthToken] as string); + + dynamic result = JObject.Parse(jsonResult); + dynamic abc = (JObject)result.result; + context[Context.FlowConfig] = (string)result.result.name; + return new StepResult(!string.IsNullOrWhiteSpace(context[Context.FlowConfig] as String), + nameof(GetFlow), "acquired flow"); + } + } +} diff --git a/Tests/ScenarioTester/.editorconfig b/Tests/ScenarioTester/.editorconfig new file mode 100644 index 000000000..ecb0c8bf9 --- /dev/null +++ b/Tests/ScenarioTester/.editorconfig @@ -0,0 +1,203 @@ +# To learn more about .editorconfig see https://aka.ms/editorconfigdocs +############################### +# Core EditorConfig Options # +############################### +# All files +[*] +indent_style = space +# Code files +[*.{cs,csx,vb,vbx}] +indent_size = 4 +insert_final_newline = true +charset = utf-8-bom +trim_trailing_whitespace = true +############################### +# .NET Coding Conventions # +############################### +[*.{cs,vb}] +# Organize usings +dotnet_sort_system_directives_first = false:error +# Language keywords vs BCL types preferences +dotnet_style_predefined_type_for_locals_parameters_members = true:error +dotnet_style_predefined_type_for_member_access = true:error +# Parentheses preferences +dotnet_style_parentheses_in_arithmetic_binary_operators = always_for_clarity:error +dotnet_style_parentheses_in_relational_binary_operators = always_for_clarity:error +dotnet_style_parentheses_in_other_binary_operators = always_for_clarity:error +dotnet_style_parentheses_in_other_operators = never_if_unnecessary:error +# Modifier preferences +dotnet_style_require_accessibility_modifiers = for_non_interface_members:error +dotnet_style_readonly_field = true:error +# Expression-level preferences +dotnet_style_object_initializer = true:error +dotnet_style_collection_initializer = true:error +dotnet_style_explicit_tuple_names = true:error +dotnet_style_null_propagation = true:error +dotnet_style_coalesce_expression = true:error +dotnet_style_prefer_is_null_check_over_reference_equality_method = true:error +dotnet_prefer_inferred_tuple_names = true:error +dotnet_prefer_inferred_anonymous_type_member_names = true:error +dotnet_style_prefer_auto_properties = true:error +dotnet_style_prefer_conditional_expression_over_assignment = false:error +dotnet_style_prefer_conditional_expression_over_return = false:error +############################### +# Naming Conventions # +############################### + +# Symbols +# constant_fields - Define constant fields +dotnet_naming_symbols.constant_fields.applicable_kinds = field +dotnet_naming_symbols.constant_fields.required_modifiers = const +# non_private_readonly_fields - Define public, internal and protected readonly fields +dotnet_naming_symbols.non_private_readonly_fields.applicable_accessibilities = public, internal, protected +dotnet_naming_symbols.non_private_readonly_fields.applicable_kinds = field +dotnet_naming_symbols.non_private_readonly_fields.required_modifiers = readonly +# static_readonly_fields - Define static and readonly fields +dotnet_naming_symbols.static_readonly_fields.applicable_kinds = field +dotnet_naming_symbols.static_readonly_fields.required_modifiers = static, readonly +# private_readonly_fields - Define private readonly fields +dotnet_naming_symbols.private_readonly_fields.applicable_accessibilities = private +dotnet_naming_symbols.private_readonly_fields.applicable_kinds = field +dotnet_naming_symbols.private_readonly_fields.required_modifiers = readonly +# public_internal_fields - Define public and internal fields +dotnet_naming_symbols.public_internal_fields.applicable_accessibilities = public, internal +dotnet_naming_symbols.public_internal_fields.applicable_kinds = field +# private_protected_fields - Define private and protected fields +dotnet_naming_symbols.private_protected_fields.applicable_accessibilities = private, protected +dotnet_naming_symbols.private_protected_fields.applicable_kinds = field +# public_symbols - Define any public symbol +dotnet_naming_symbols.public_symbols.applicable_accessibilities = public, internal, protected, protected_internal +dotnet_naming_symbols.public_symbols.applicable_kinds = method, property, event, delegate +# parameters - Defines any parameter +dotnet_naming_symbols.parameters.applicable_kinds = parameter +# non_interface_types - Defines class, struct, enum and delegate types +dotnet_naming_symbols.non_interface_types.applicable_kinds = class, struct, enum, delegate +# interface_types - Defines interfaces +dotnet_naming_symbols.interface_types.applicable_kinds = interface +# private fields - Define private fields +dotnet_naming_symbols.private_fields.applicable_kinds = field +dotnet_naming_symbols.private_fields.applicable_accessibilities = private +# private static fields - Define private static fields, includes const since it is implicity static +dotnet_naming_symbols.private_static_fields.applicable_kinds = field +dotnet_naming_symbols.private_static_fields.applicable_accessibilities = private +dotnet_naming_symbols.private_static_fields.required_modifiers = static + +# Styles +# camel_case - Define the camelCase style +dotnet_naming_style.camel_case.capitalization = camel_case +# pascal_case - Define the Pascal_case style +dotnet_naming_style.pascal_case.capitalization = pascal_case +# first_upper - The first character must start with an upper-case character +dotnet_naming_style.first_upper.capitalization = first_word_upper +# prefix_interface_interface_with_i - Interfaces must be PascalCase and the first character of an interface must be an 'I' +dotnet_naming_style.prefix_interface_interface_with_i.capitalization = pascal_case +dotnet_naming_style.prefix_interface_interface_with_i.required_prefix = I +# start_with_underscore - Define start with underscore style +dotnet_naming_style.start_with_underscore.capitalization = camel_case +dotnet_naming_style.start_with_underscore.required_prefix = _ +# start_with_underscore_pascal_case - Define start with underscore style +dotnet_naming_style.start_with_underscore_pascal_case.capitalization = pascal_case +dotnet_naming_style.start_with_underscore_pascal_case.required_prefix = _ + +# Rules +# Private static and const must start with underscore and PascalCase +dotnet_naming_rule.fields_must_start_with_underscore_static.severity = error +dotnet_naming_rule.fields_must_start_with_underscore_static.symbols = private_static_fields +dotnet_naming_rule.fields_must_start_with_underscore_static.style = start_with_underscore_pascal_case +# Private fields must start with underscore (fields_must_start_with_underscore) +dotnet_naming_rule.private_fields_must_start_with_underscore.severity = error +dotnet_naming_rule.private_fields_must_start_with_underscore.symbols = private_fields +dotnet_naming_rule.private_fields_must_start_with_underscore.style = start_with_underscore +# Constant fields must be PascalCase +dotnet_naming_rule.constant_fields_must_be_pascal_case.severity = error +dotnet_naming_rule.constant_fields_must_be_pascal_case.symbols = constant_fields +dotnet_naming_rule.constant_fields_must_be_pascal_case.style = pascal_case +# Static readonly fields must be PascalCase +dotnet_naming_rule.static_readonly_fields_must_be_pascal_case.severity = error +dotnet_naming_rule.static_readonly_fields_must_be_pascal_case.symbols = static_readonly_fields +dotnet_naming_rule.static_readonly_fields_must_be_pascal_case.style = pascal_case +# Public, internal and protected readonly fields must be PascalCase +dotnet_naming_rule.non_private_readonly_fields_must_be_pascal_case.severity = error +dotnet_naming_rule.non_private_readonly_fields_must_be_pascal_case.symbols = non_private_readonly_fields +dotnet_naming_rule.non_private_readonly_fields_must_be_pascal_case.style = pascal_case +# Public and internal fields must be PascalCase +dotnet_naming_rule.public_internal_fields_must_be_pascal_case.severity = error +dotnet_naming_rule.public_internal_fields_must_be_pascal_case.symbols = public_internal_fields +dotnet_naming_rule.public_internal_fields_must_be_pascal_case.style = pascal_case +# Private and protected fields must be camelCase +dotnet_naming_rule.private_protected_fields_must_be_camel_case.severity = error +dotnet_naming_rule.private_protected_fields_must_be_camel_case.symbols = private_protected_fields +dotnet_naming_rule.private_protected_fields_must_be_camel_case.style = camel_case +# Public members must be capitalized +dotnet_naming_rule.public_members_must_be_capitalized.severity = error +dotnet_naming_rule.public_members_must_be_capitalized.symbols = public_symbols +dotnet_naming_rule.public_members_must_be_capitalized.style = first_upper +# Parameters must be camelCase +dotnet_naming_rule.parameters_must_be_camel_case.severity = error +dotnet_naming_rule.parameters_must_be_camel_case.symbols = parameters +dotnet_naming_rule.parameters_must_be_camel_case.style = camel_case +# Class, struct, enum and delegates must be PascalCase +dotnet_naming_rule.non_interface_types_must_be_pascal_case.severity = error +dotnet_naming_rule.non_interface_types_must_be_pascal_case.symbols = non_interface_types +dotnet_naming_rule.non_interface_types_must_be_pascal_case.style = pascal_case +# Interfaces must be PascalCase and start with an 'I' +dotnet_naming_rule.interface_types_must_be_prefixed_with_i.severity = error +dotnet_naming_rule.interface_types_must_be_prefixed_with_i.symbols = interface_types +dotnet_naming_rule.interface_types_must_be_prefixed_with_i.style = prefix_interface_interface_with_i + + +############################### +# C# Coding Conventions # +############################### +[*.cs] +# Expression-bodied members +csharp_style_expression_bodied_methods = false:error +csharp_style_expression_bodied_constructors = false:error +csharp_style_expression_bodied_operators = false:error +csharp_style_expression_bodied_properties = true:error +csharp_style_expression_bodied_indexers = true:error +csharp_style_expression_bodied_accessors = true:error +# Pattern matching preferences +csharp_style_pattern_matching_over_is_with_cast_check = true:error +csharp_style_pattern_matching_over_as_with_null_check = true:error +# Null-checking preferences +csharp_style_throw_expression = true:error +csharp_style_conditional_delegate_call = true:error +# Modifier preferences +csharp_preferred_modifier_order = public,private,protected,internal,static,extern,new,virtual,abstract,sealed,override,readonly,unsafe,volatile,async:error +# Expression-level preferences +csharp_prefer_braces = true:error +csharp_style_deconstructed_variable_declaration = true:error +csharp_prefer_simple_default_expression = true:error +csharp_style_pattern_local_over_anonymous_function = true:error +csharp_style_inlined_variable_declaration = true:error +############################### +# C# Formatting Rules # +############################### +# New line preferences +csharp_new_line_before_open_brace = all +csharp_new_line_before_else = true +csharp_new_line_before_catch = true +csharp_new_line_before_finally = true +csharp_new_line_before_members_in_object_initializers = true +csharp_new_line_before_members_in_anonymous_types = true +csharp_new_line_between_query_expression_clauses = true +# Indentation preferences +csharp_indent_case_contents = true +csharp_indent_switch_labels = true +csharp_indent_labels = one_less_than_current +# Space preferences +csharp_space_after_cast = false +csharp_space_after_keywords_in_control_flow_statements = true +csharp_space_between_method_call_parameter_list_parentheses = false +csharp_space_between_method_declaration_parameter_list_parentheses = false +csharp_space_between_parentheses = false +csharp_space_before_colon_in_inheritance_clause = true +csharp_space_after_colon_in_inheritance_clause = true +csharp_space_around_binary_operators = before_and_after +csharp_space_between_method_declaration_empty_parameter_list_parentheses = false +csharp_space_between_method_call_name_and_opening_parenthesis = false +csharp_space_between_method_call_empty_parameter_list_parentheses = false +# Wrapping preferences +csharp_preserve_single_line_statements = false +csharp_preserve_single_line_blocks = true diff --git a/Tests/ScenarioTester/README.md b/Tests/ScenarioTester/README.md new file mode 100644 index 000000000..5d877af88 --- /dev/null +++ b/Tests/ScenarioTester/README.md @@ -0,0 +1,58 @@ +## Scenario Tester + +Scenario Tester is designed to run through a number of actions in a sequence with N number of parallel executions of the scenario. + +To use it: + +1. Create a set of "ScenarioSteps" identified by the [StepAttribute] that takes the ScenarioContext and returns a StepResult. Each Scenario Step is an autonomous function +that gets all its inputs from the scenario context, executes the action for the step, validates the action results, and returns the result of the step in StepResult. Any outputs +can be added into the ScenarioContext to be passed down to the next step in the Scenario chain. + +``` +public class Host +{ + [Step("createResource")] + public static StepResult CreateResource(ScenarioContext context) + { + // get step input from the context + var resourceName = context["resource_name"] + + // run the step + string jsonResult = Request.Get(serverAddress + "?resource=" + resourceName); + + // add the output of the result to the context + dynamic result = JsonConvert.DeserializeObject(jsonResult); + context["resource_id"] = result.resource_id; + + // validate the step output + return new StepResult(result.resource_id != null, "resource was created"); + } +} +``` + +2. A scenario is a collection of steps. + +``` +// using the steps directly +var scenario = new ScenarioDescription("resourceAddDelete", + Host.CreateResource, + Host.DeleteResource); + +// or from JSON +var scenario = ScenarioDescription.FromJson("resourceAddDelete", @" +[ + {'action':'createResource'}, + {'action':'deleteResource'} +]", typeof(Host)); +``` + +3. The scenario can then be run. + +``` +var context = new ScenarioContext(); +context["resource_name"] = "ResourceA"; + +var results = await ScenarioResult.RunAsync(scenario, context, 100); +``` + + diff --git a/Tests/ScenarioTester/ScenarioTester.nuspec b/Tests/ScenarioTester/ScenarioTester.nuspec new file mode 100644 index 000000000..8fbc740b3 --- /dev/null +++ b/Tests/ScenarioTester/ScenarioTester.nuspec @@ -0,0 +1,21 @@ + + + + Microsoft.ScenarioTester + $version$ + Microsoft + http://aka.ms/data-accelerator + MIT + https://raw.githubusercontent.com/wiki/Microsoft/data-accelerator/tutorials/images/roundwhite6464.PNG + true + Enables running scenarios against a server or application. + © Microsoft Corporation. All rights reserved. + ScenarioTester + + + + + + + + \ No newline at end of file diff --git a/Tests/ScenarioTester/ScenarioTester.sln b/Tests/ScenarioTester/ScenarioTester.sln new file mode 100644 index 000000000..8f6accfa2 --- /dev/null +++ b/Tests/ScenarioTester/ScenarioTester.sln @@ -0,0 +1,37 @@ + +Microsoft Visual Studio Solution File, Format Version 12.00 +# Visual Studio Version 16 +VisualStudioVersion = 16.0.28803.156 +MinimumVisualStudioVersion = 10.0.40219.1 +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "ScenarioTester", "ScenarioTester\ScenarioTester.csproj", "{AF410B87-CD2C-4C19-947D-CCE64EC0C7DD}" +EndProject +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "ScenarioTesterTests", "ScenarioTesterTests\ScenarioTesterTests.csproj", "{798D1032-BE6D-4FE1-8692-B238522A5ACE}" +EndProject +Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Solution Items", "Solution Items", "{E2391215-8738-4067-A35E-6AA6667F6DB2}" + ProjectSection(SolutionItems) = preProject + README.md = README.md + ScenarioTester.nuspec = ScenarioTester.nuspec + EndProjectSection +EndProject +Global + GlobalSection(SolutionConfigurationPlatforms) = preSolution + Debug|Any CPU = Debug|Any CPU + Release|Any CPU = Release|Any CPU + EndGlobalSection + GlobalSection(ProjectConfigurationPlatforms) = postSolution + {AF410B87-CD2C-4C19-947D-CCE64EC0C7DD}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {AF410B87-CD2C-4C19-947D-CCE64EC0C7DD}.Debug|Any CPU.Build.0 = Debug|Any CPU + {AF410B87-CD2C-4C19-947D-CCE64EC0C7DD}.Release|Any CPU.ActiveCfg = Release|Any CPU + {AF410B87-CD2C-4C19-947D-CCE64EC0C7DD}.Release|Any CPU.Build.0 = Release|Any CPU + {798D1032-BE6D-4FE1-8692-B238522A5ACE}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {798D1032-BE6D-4FE1-8692-B238522A5ACE}.Debug|Any CPU.Build.0 = Debug|Any CPU + {798D1032-BE6D-4FE1-8692-B238522A5ACE}.Release|Any CPU.ActiveCfg = Release|Any CPU + {798D1032-BE6D-4FE1-8692-B238522A5ACE}.Release|Any CPU.Build.0 = Release|Any CPU + EndGlobalSection + GlobalSection(SolutionProperties) = preSolution + HideSolutionNode = FALSE + EndGlobalSection + GlobalSection(ExtensibilityGlobals) = postSolution + SolutionGuid = {1FAF91B3-08FA-49AB-BEE7-B5FB324B4C30} + EndGlobalSection +EndGlobal diff --git a/Tests/ScenarioTester/ScenarioTester/DontTimeAttribute.cs b/Tests/ScenarioTester/ScenarioTester/DontTimeAttribute.cs new file mode 100644 index 000000000..8ccb2bda2 --- /dev/null +++ b/Tests/ScenarioTester/ScenarioTester/DontTimeAttribute.cs @@ -0,0 +1,16 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using System; + +namespace ScenarioTester +{ + /// + /// Attribute to identity which steps to skip time reporting on. + /// + internal class DontTimeAttribute : Attribute + { + } +} diff --git a/Tests/ScenarioTester/ScenarioTester/FinalPublicKey.snk b/Tests/ScenarioTester/ScenarioTester/FinalPublicKey.snk new file mode 100644 index 000000000..110b59c7b Binary files /dev/null and b/Tests/ScenarioTester/ScenarioTester/FinalPublicKey.snk differ diff --git a/Tests/ScenarioTester/ScenarioTester/Request.cs b/Tests/ScenarioTester/ScenarioTester/Request.cs new file mode 100644 index 000000000..fb1fe49f3 --- /dev/null +++ b/Tests/ScenarioTester/ScenarioTester/Request.cs @@ -0,0 +1,187 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +using Newtonsoft.Json; +using System; +using System.Collections.Generic; +using System.IO; +using System.Net; +using System.Text; +using static System.FormattableString; + +namespace ScenarioTester +{ + /// + /// Content to be sent to the Request + /// + public class RequestContent + { + public readonly string ContentType; + public readonly byte[] Data; + + /// + /// Create content for a request. + /// + /// data in bytes + /// type of content + public RequestContent(byte[] data, string contentType) + { + Data = data; + ContentType = contentType; + } + + /// + /// Create a as a json paylod. + /// + /// The object to encode as json + /// optional content type; defaults to 'application/json' + /// + public static RequestContent EncodeAsJson(object payload, string contentType = "application/json") + { + return new RequestContent( + Encoding.UTF8.GetBytes(JsonConvert.SerializeObject(payload)), + contentType + ); + } + } + + /// + /// Utilities for handling requests to an API server. + /// + public class Request + { + /// + /// Default request timeout of 20 minutes + /// + private const int _RequestTimeout = 1000 * 60 * 20; + + /// + /// Helper function to do an HTTP GET. + /// + /// URL to GET response from + /// Optional bearer token to pass to the request Auth header + /// The response for the request as a string + public static string Get(string url, string bearerToken = null) + { + WebRequest req = WebRequest.Create(url); + req.Method = "GET"; + req.Timeout = _RequestTimeout; + if (bearerToken != null) + { + req.Headers.Add("Authorization", $"Bearer {bearerToken}"); + } + req.Headers.Add("Content-type", "application/json"); + + try + { + var resp = req.GetResponse(); + using (var reader = new StreamReader(resp.GetResponseStream())) + { + return reader.ReadToEnd(); + } + } + catch (WebException we) + { + string returned; + if (we.Response is HttpWebResponse) + { + returned = "Server returned " + ((HttpWebResponse)we.Response).StatusCode; + } + else + { + returned = "Response is " + we.Response; + } + + throw new Exception( + Invariant($"Error (${we.Message}) posting to: {url} {returned} {GetErrorResponse(we)}"), + we + ); + } + } + + /// + /// Helper function to do an HTTP POST with an optional set of headers to set. + /// + /// URL to POST request to + /// to send to the server + /// Optional bearer token to pass to the request Auth header + /// Any additional headers to add to the request + /// The response for the request as a string + public static string Post(string url, RequestContent content, string bearerToken = null, Dictionary additionalHeaders = null) + { + return PostPut("POST", url, content, bearerToken, additionalHeaders); + } + + /// + /// Helper function to do an HTTP PUT with an optional set of headers to set. + /// + /// URL to PUT request to + /// to send to the server + /// Optional bearer token to pass to the request Auth header + /// Any additional headers to add to the request + /// The response for the request as a string + public static string Put(string url, RequestContent content, string bearerToken = null, Dictionary additionalHeaders = null) + { + return PostPut("PUT", url, content, bearerToken, additionalHeaders); + } + + private static string PostPut(string requestMethod, string url, RequestContent content, string bearerToken = null, Dictionary additionalHeaders = null) + { + var req = (HttpWebRequest)WebRequest.Create(url); + req.Method = requestMethod; + req.Timeout = _RequestTimeout; + req.ContentLength = content.Data.Length; + req.ContentType = content.ContentType; + if (bearerToken != null) + { + req.Headers.Add("Authorization", $"Bearer {bearerToken}"); + } + if (additionalHeaders != null && additionalHeaders.Count > 0) + { + foreach (var headerValue in additionalHeaders) + { + req.Headers.Add(headerValue.Key, headerValue.Value); + } + } + + using (var reqStream = req.GetRequestStream()) + { + reqStream.Write(content.Data, 0, content.Data.Length); + } + try + { + var resp = req.GetResponse(); + using (var reader = new StreamReader(resp.GetResponseStream())) + { + return reader.ReadToEnd(); + } + } + catch (WebException we) + { + string returned; + if (we.Response is HttpWebResponse) + { + returned = "Server returned " + ((HttpWebResponse)we.Response).StatusCode; + } + else + { + returned = "Response is " + we.Response; + } + + throw new Exception( + Invariant($"Error (${we.Message}) posting to: {url} {returned} {GetErrorResponse(we)}"), + we + ); + } + } + + private static string GetErrorResponse(WebException we) + { + using (var reader = new StreamReader(we.Response.GetResponseStream())) + { + return reader.ReadToEnd(); + } + } + } +} diff --git a/Tests/ScenarioTester/ScenarioTester/ScenarioContext.cs b/Tests/ScenarioTester/ScenarioTester/ScenarioContext.cs new file mode 100644 index 000000000..888ed3d2b --- /dev/null +++ b/Tests/ScenarioTester/ScenarioTester/ScenarioContext.cs @@ -0,0 +1,40 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using System; +using System.Collections.Concurrent; +using System.Linq; + +namespace ScenarioTester +{ + /// + /// Shared context for all the scenario steps. Could be used to pass in parameters + /// as input and output for steps. + /// + public class ScenarioContext : ConcurrentDictionary, IDisposable + { + public ScenarioContext() : base() { } + + /// + /// Copy another context + /// + public ScenarioContext(ScenarioContext context) + : base(context.ToDictionary(c => c.Key, c => c.Value)) { } + + /// + /// Dispose of all disposable properties. + /// + public void Dispose() + { + foreach (var value in this.Values) + { + if (value is IDisposable disposable) + { + disposable.Dispose(); + } + } + } + } +} diff --git a/Tests/ScenarioTester/ScenarioTester/ScenarioDescription.cs b/Tests/ScenarioTester/ScenarioTester/ScenarioDescription.cs new file mode 100644 index 000000000..3c8f9f530 --- /dev/null +++ b/Tests/ScenarioTester/ScenarioTester/ScenarioDescription.cs @@ -0,0 +1,81 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using Newtonsoft.Json; +using System; +using System.Collections.Generic; +using System.Linq; +using System.Reflection; + +namespace ScenarioTester +{ + /// + /// Describes a scenario of different "actions" sharing the same context. + /// + public class ScenarioDescription + { + /// + /// Description of the scenario + /// + public string Description { get; } + + /// + /// Create a Scenario description from a sequence of actions + /// + /// a description of the scenario + /// sequence of steps to execute + public ScenarioDescription(string description, params ScenarioStep[] steps) + { + this.Description = description; + this.Steps = steps; + } + + /// + /// Sequence of steps defining a scenario + /// + public IReadOnlyCollection Steps { get; } + + /// + /// Create from using + /// steps defined in . + /// + /// A scenario description of steps in json + public static ScenarioDescription FromJson(string description, string json, params Type[] types) + { + var definedSteps = GetStepDefinitions(types); + var steps = JsonConvert.DeserializeObject(json); + + var scenarioSteps = new List(); + for (int i = 0; i < steps.Length; i++) + { + var found = definedSteps.FirstOrDefault( + s => s.GetMethodInfo().GetCustomAttribute().Name == steps[i].Action); + + if (found == null) + { + throw new Exception($"{steps[i].Action} not found in Step definitions"); + } + scenarioSteps.Add(found); + } + + var scenario = new ScenarioDescription(description, scenarioSteps.ToArray()); + return scenario; + } + + private static IEnumerable GetStepDefinitions(params Type[] types) + { + return types + .SelectMany(x => x.GetMethods(BindingFlags.Static | BindingFlags.Public)) + .Where(y => y.GetCustomAttributes().OfType().Any()) + .Select(method => (ScenarioStep)Delegate.CreateDelegate(typeof(ScenarioStep), method)); + } + + private class StepDefinition + { + [JsonProperty("action")] + public string Action; + } + } +} diff --git a/Tests/ScenarioTester/ScenarioTester/ScenarioResult.cs b/Tests/ScenarioTester/ScenarioTester/ScenarioResult.cs new file mode 100644 index 000000000..a2ca5d406 --- /dev/null +++ b/Tests/ScenarioTester/ScenarioTester/ScenarioResult.cs @@ -0,0 +1,99 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using System; +using System.Collections.Generic; +using System.Diagnostics; +using System.Reflection; +using System.Threading.Tasks; + +namespace ScenarioTester +{ + /// + /// Executes a scenario and provides the results for the scenario. + /// + public class ScenarioResult + { + private static readonly Stopwatch _Watch = Stopwatch.StartNew(); + + /// + /// True if any of the steps in the scenario execution failed. + /// + public bool Failed { get; private set; } + + public ScenarioResult(ScenarioDescription scenario) + { + Scenario = scenario; + } + + /// + /// Result of each step in the + /// + public List StepResults { get; } = new List(); + + /// + /// The scenario definition to execute + /// + public ScenarioDescription Scenario { get; } + + /// + /// Run the scenario using the given . + /// + public void Run(ScenarioContext context) + { + // TODO: Log the start-end time. + var startTime = _Watch.ElapsedMilliseconds; + + foreach (var step in this.Scenario.Steps) + { + var stepStart = _Watch.ElapsedMilliseconds; + + try + { + var result = step(context); + this.StepResults.Add(result); + var stepEnd = _Watch.ElapsedMilliseconds; + this.Failed |= !result.Success; + } + catch (Exception e) + { + this.Failed = true; + + var stepName = step.Method.GetCustomAttribute().Name; + this.StepResults.Add(new StepResult(false, stepName, exception: e)); + var stepEnd = _Watch.ElapsedMilliseconds; + // break; Don't fail but continue to allow any cleanups to happen. + // TODO: make this configurable + } + } + + var endTime = _Watch.ElapsedMilliseconds; + } + + /// + /// Run the given for number of iterations parallalelly. + /// Each iteration will get their own iteration context that will be shared across the scenario steps. + /// + /// Collection of all . + public static async Task> RunAsync(ScenarioDescription scenario, ScenarioContext context, int count) + { + List tasks = new List(); + List results = new List(); + + for (int i = 0; i < count; i++) + { + using (var iterationContext = new ScenarioContext(context)) + { + var result = new ScenarioResult(scenario); + results.Add(result); + tasks.Add(Task.Run(() => result.Run(iterationContext))); + } + } + + await Task.WhenAll(tasks.ToArray()); + return results; + } + } +} diff --git a/Tests/ScenarioTester/ScenarioTester/ScenarioTester.csproj b/Tests/ScenarioTester/ScenarioTester/ScenarioTester.csproj new file mode 100644 index 000000000..7c093df89 --- /dev/null +++ b/Tests/ScenarioTester/ScenarioTester/ScenarioTester.csproj @@ -0,0 +1,24 @@ + + + + netcoreapp2.2 + ScenarioTester + ScenarioTester + + + + + all + runtime; build; native; contentfiles; analyzers + + + + + + + Microsoft400 + StrongName + + + + diff --git a/Tests/ScenarioTester/ScenarioTester/StepAttribute.cs b/Tests/ScenarioTester/ScenarioTester/StepAttribute.cs new file mode 100644 index 000000000..e1ea5dd2b --- /dev/null +++ b/Tests/ScenarioTester/ScenarioTester/StepAttribute.cs @@ -0,0 +1,28 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using System; + +namespace ScenarioTester +{ + /// + /// Identifies a scenario step method. Must be static + /// + [AttributeUsage(AttributeTargets.Method, Inherited = false, AllowMultiple = false)] + public class StepAttribute : Attribute + { + public StepAttribute(string stepName) + { + this.Name = stepName; + } + + /// + /// Step name + /// + public string Name { get; } + } + + public delegate StepResult ScenarioStep(ScenarioContext context); +} diff --git a/Tests/ScenarioTester/ScenarioTester/StepResult.cs b/Tests/ScenarioTester/ScenarioTester/StepResult.cs new file mode 100644 index 000000000..39ab29614 --- /dev/null +++ b/Tests/ScenarioTester/ScenarioTester/StepResult.cs @@ -0,0 +1,51 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using System; + +namespace ScenarioTester +{ + /// + /// Result of the execution. + /// + public class StepResult + { + /// + /// True if the scenario step was successful + /// + public readonly bool Success; + + /// + /// Description of the step result + /// + public readonly string Description; + + /// + /// Value of the result. + /// + public readonly string Value; + + /// + /// If there was an exception while executing the step. + /// + public readonly Exception Exception; + + /// + /// Create a step result for the + /// + /// True if the scenario step was successful + /// Description of the step result + /// Value of the result + /// If there was an exception while executing the step. + public StepResult(bool success, string description, string result = null, Exception exception = null) + { + Success = success; + Description = description; + Value = result; + Exception = exception; + } + } + +} diff --git a/Tests/ScenarioTester/ScenarioTesterTests/ScenarioTesterTests.csproj b/Tests/ScenarioTester/ScenarioTesterTests/ScenarioTesterTests.csproj new file mode 100644 index 000000000..c771a592f --- /dev/null +++ b/Tests/ScenarioTester/ScenarioTesterTests/ScenarioTesterTests.csproj @@ -0,0 +1,19 @@ + + + + netcoreapp2.2 + + false + + + + + + + + + + + + + diff --git a/Tests/ScenarioTester/ScenarioTesterTests/TestScenarioTester.cs b/Tests/ScenarioTester/ScenarioTesterTests/TestScenarioTester.cs new file mode 100644 index 000000000..014112980 --- /dev/null +++ b/Tests/ScenarioTester/ScenarioTesterTests/TestScenarioTester.cs @@ -0,0 +1,158 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +using ScenarioTester; +using Microsoft.VisualStudio.TestTools.UnitTesting; +using System.Linq; + +namespace TestScenarioTester +{ + [TestClass] + public class TestScenarioTester + { + public class TestHost + { + public static StepResult Step1(ScenarioContext context) + { + return new StepResult(true, nameof(Step1)); + } + + [Step(nameof(Step2))] + public static StepResult Step2(ScenarioContext context) + { + int sequence = 0; + if (context.TryGetValue("sequence", out object sequenceObj)) + { + sequence = (int)sequenceObj; + } + context[nameof(Step2)] = sequence++; + context["sequence"] = sequence; + + return new StepResult(true, nameof(Step2)); + } + + [Step(nameof(FailedStep))] + public static StepResult FailedStep(ScenarioContext context) + { + int sequence = 0; + if (context.TryGetValue("sequence", out object sequenceObj)) + { + sequence = (int)sequenceObj; + } + context[nameof(FailedStep)] = sequence++; + context["sequence"] = sequence; + + return new StepResult(false, nameof(FailedStep)); + } + } + + [TestMethod] + public void CreateScenarioFromJson() + { + var scenario = ScenarioDescription.FromJson("scenario", @"[{'action':'Step2'}]", typeof(TestHost)); + Assert.AreEqual(1, scenario.Steps.Count); + Assert.AreEqual(nameof(TestHost.Step2), scenario.Steps.First().Method.Name); + } + + [TestMethod] + public void ScenarioFailsWithFailedStep() + { + var scenario = new ScenarioDescription("scenario", TestHost.FailedStep, TestHost.Step2); + var result = new ScenarioResult(scenario); + + result.Run(new ScenarioContext()); + + Assert.IsTrue(result.Failed); + } + + [TestMethod] + public void ScenarioPassesWithStep() + { + var scenario = new ScenarioDescription("scenario", TestHost.Step2); + var result = new ScenarioResult(scenario); + + result.Run(new ScenarioContext()); + + Assert.IsFalse(result.Failed); + } + + [TestMethod] + public void ExecutionSequenceByMethod() + { + var scenario = new ScenarioDescription("scenario1", TestHost.FailedStep, TestHost.Step2); + var result = new ScenarioResult(scenario); + + var testContext = new ScenarioContext(); + testContext["sequence"] = 1; + result.Run(testContext); + + Assert.AreEqual(1, (int)testContext[nameof(TestHost.FailedStep)], scenario.Description); + Assert.AreEqual(2, (int)testContext[nameof(TestHost.Step2)], scenario.Description); + + // change sequence and make sure that it is different this time + scenario = new ScenarioDescription("scenario2", TestHost.Step2, TestHost.FailedStep); + result = new ScenarioResult(scenario); + + testContext = new ScenarioContext(); + testContext["sequence"] = 1; + result.Run(testContext); + + Assert.AreEqual(2, (int)testContext[nameof(TestHost.FailedStep)], scenario.Description); + Assert.AreEqual(1, (int)testContext[nameof(TestHost.Step2)], scenario.Description); + + } + + [TestMethod] + public void ExecutionSequenceByJson() + { + var scenario = ScenarioDescription.FromJson("scenario1", @"[{'action':'Step2'}, {'action':'FailedStep'}]", typeof(TestHost)); + var result = new ScenarioResult(scenario); + + var testContext = new ScenarioContext(); + testContext["sequence"] = 1; + result.Run(testContext); + + Assert.AreEqual(2, (int)testContext[nameof(TestHost.FailedStep)], scenario.Description); + Assert.AreEqual(1, (int)testContext[nameof(TestHost.Step2)], scenario.Description); + + // change sequence and make sure that it is different this time + scenario = ScenarioDescription.FromJson("scenario2", @"[{'action':'FailedStep'}, {'action':'Step2'}]", typeof(TestHost)); + result = new ScenarioResult(scenario); + + testContext = new ScenarioContext(); + testContext["sequence"] = 1; + result.Run(testContext); + + Assert.AreEqual(1, (int)testContext[nameof(TestHost.FailedStep)], scenario.Description); + Assert.AreEqual(2, (int)testContext[nameof(TestHost.Step2)], scenario.Description); + } + + [TestMethod] + public void ContextIsDuplicated() + { + var context1 = new ScenarioContext(); + context1["init"] = "init"; + + var context2 = new ScenarioContext(context1); + + Assert.AreEqual(1, context2.Count, "count of keys copied over"); + Assert.AreEqual("init", context2["init"], "value copied over"); + + context1["newincontext1"] = "newincontext1"; + + Assert.AreEqual(1, context2.Count, "count is not changed when context1 changes"); + Assert.IsFalse(context2.ContainsKey("newincontext1"), "new key in context1 is not in context2"); + + context2["newincontext2"] = "newincontext2"; + + Assert.IsFalse(context1.ContainsKey("newincontext2"), "new key in context2 is not in context1"); + + context2["init"] = "changed"; + + Assert.AreEqual("changed", context2["init"], "initial keys updated when changed in context"); + Assert.AreEqual("init", context1["init"], "initial keys not updated when changed in another context"); + } + } +} diff --git a/Website/Packages/datax-common/localdevpatch.bat b/Website/Packages/datax-common/localdevpatch.bat index 2ad5a7d05..b5cb578f4 100644 --- a/Website/Packages/datax-common/localdevpatch.bat +++ b/Website/Packages/datax-common/localdevpatch.bat @@ -21,6 +21,9 @@ xcopy dist ..\datax-metrics\node_modules\datax-common\dist /i /s /y REM Patching dist folder to jobs xcopy dist ..\datax-jobs\node_modules\datax-common\dist /i /s /y +REM Patching dist folder to query +xcopy dist ..\datax-query\node_modules\datax-common\dist /i /s /y + @echo off REM Patch all other dist folders that you want to patch here REM For example, you may want to patch the feature packages that uses this package \ No newline at end of file diff --git a/Website/Packages/datax-common/package.json b/Website/Packages/datax-common/package.json index 9877c5465..929485ccf 100644 --- a/Website/Packages/datax-common/package.json +++ b/Website/Packages/datax-common/package.json @@ -1,6 +1,6 @@ { "name": "datax-common", - "version": "1.2.2", + "version": "1.3.0", "description": "Common UX, styles, utilities, and modules", "author": "Microsoft", "license": "MIT", diff --git a/Website/Packages/datax-pipeline/src/common/apiConstants.js b/Website/Packages/datax-common/src/common/apiConstants.js similarity index 93% rename from Website/Packages/datax-pipeline/src/common/apiConstants.js rename to Website/Packages/datax-common/src/common/apiConstants.js index 9cd935296..4ce2ab850 100644 --- a/Website/Packages/datax-pipeline/src/common/apiConstants.js +++ b/Website/Packages/datax-common/src/common/apiConstants.js @@ -20,7 +20,7 @@ export const Constants = { export const ApiNames = { FunctionEnabled: 'functionenabled', - + IsDatabricksSparkType: 'isdatabrickssparktype', // OneBox EnableLocalOneBox: 'enableLocalOneBox' }; diff --git a/Website/Packages/datax-common/src/common/index.js b/Website/Packages/datax-common/src/common/index.js new file mode 100644 index 000000000..a2aacb0f2 --- /dev/null +++ b/Website/Packages/datax-common/src/common/index.js @@ -0,0 +1,11 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +/** + * Common - Specify here all the common modules you want to expose externally + */ +import { Constants, ApiNames } from './apiConstants'; + +export { Constants, ApiNames }; diff --git a/Website/Packages/datax-common/src/index.js b/Website/Packages/datax-common/src/index.js index 54f4d46c6..189d1ea6b 100644 --- a/Website/Packages/datax-common/src/index.js +++ b/Website/Packages/datax-common/src/index.js @@ -13,3 +13,4 @@ export * from './components'; export * from './modules'; export * from './styles'; export * from './utils'; +export * from './common'; diff --git a/Website/Packages/datax-common/src/utils/helpers.js b/Website/Packages/datax-common/src/utils/helpers.js new file mode 100644 index 000000000..503f47528 --- /dev/null +++ b/Website/Packages/datax-common/src/utils/helpers.js @@ -0,0 +1,11 @@ + +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +export function isValidNumberAboveZero(value) { + const number = Number(value); + const isNumber = !isNaN(number); + const isValid = isNumber && number > 0 && value[0] !== '0'; + return isValid; +} \ No newline at end of file diff --git a/Website/Packages/datax-common/src/utils/index.js b/Website/Packages/datax-common/src/utils/index.js index 01341bba2..775c778ac 100644 --- a/Website/Packages/datax-common/src/utils/index.js +++ b/Website/Packages/datax-common/src/utils/index.js @@ -12,3 +12,7 @@ export * from './serviceApi'; // General common utilities export * from './utilities'; + +// General common helpers +import * as CommonHelpers from './helpers'; +export { CommonHelpers }; \ No newline at end of file diff --git a/Website/Packages/datax-home/package.json b/Website/Packages/datax-home/package.json index 279a21d8e..d7023ba44 100644 --- a/Website/Packages/datax-home/package.json +++ b/Website/Packages/datax-home/package.json @@ -1,6 +1,6 @@ { "name": "datax-home", - "version": "1.2.2", + "version": "1.3.0", "description": "Home page", "author": "Microsoft", "license": "MIT", @@ -30,7 +30,7 @@ "css-loader": "1.0.1", "file-loader": "2.0.0", "mini-css-extract-plugin": "0.4.4", - "datax-common": "1.2.2", + "datax-common": "1.3.0", "office-ui-fabric-react": "6.111.2", "q": "1.5.1", "react": "16.6.3", diff --git a/Website/Packages/datax-jobs/package.json b/Website/Packages/datax-jobs/package.json index 26d58ee9b..82feef6ed 100644 --- a/Website/Packages/datax-jobs/package.json +++ b/Website/Packages/datax-jobs/package.json @@ -1,6 +1,6 @@ { "name": "datax-jobs", - "version": "1.2.2", + "version": "1.3.0", "description": "Job features", "author": "Microsoft", "license": "MIT", @@ -30,7 +30,7 @@ "css-loader": "1.0.1", "file-loader": "2.0.0", "mini-css-extract-plugin": "0.4.4", - "datax-common": "1.2.2", + "datax-common": "1.3.0", "office-ui-fabric-react": "6.111.2", "q": "1.5.1", "prop-types": "15.6.2", diff --git a/Website/Packages/datax-jobs/src/modules/jobs/components/sparkJobs.jsx b/Website/Packages/datax-jobs/src/modules/jobs/components/sparkJobs.jsx index bd08559ea..027d0d3c7 100644 --- a/Website/Packages/datax-jobs/src/modules/jobs/components/sparkJobs.jsx +++ b/Website/Packages/datax-jobs/src/modules/jobs/components/sparkJobs.jsx @@ -58,7 +58,7 @@ class SparkJobs extends React.Component { } return ( - +
{details}
- +
); } } @@ -134,6 +134,13 @@ const mapStateToProps = state => ({}); const mapDispatchToProps = () => ({}); // Styles +const rootStyle = { + display: 'flex', + flexDirection: 'column', + overflowX: 'hidden', + overflowY: 'auto' +}; + const contentStyle = { paddingLeft: 30, paddingRight: 30, diff --git a/Website/Packages/datax-metrics/package.json b/Website/Packages/datax-metrics/package.json index ff4e68be4..3857627d6 100644 --- a/Website/Packages/datax-metrics/package.json +++ b/Website/Packages/datax-metrics/package.json @@ -1,6 +1,6 @@ { "name": "datax-metrics", - "version": "1.2.2", + "version": "1.3.0", "description": "Metric features", "author": "Microsoft", "license": "MIT", @@ -33,7 +33,7 @@ "d3": "4.10.2", "file-loader": "2.0.0", "mini-css-extract-plugin": "0.4.4", - "datax-common": "1.2.2", + "datax-common": "1.3.0", "office-ui-fabric-react": "6.111.2", "q": "1.5.1", "plotly.js": "1.44.4", diff --git a/Website/Packages/datax-metrics/src/modules/metrics/apiConstants.js b/Website/Packages/datax-metrics/src/modules/metrics/apiConstants.js index f77851a42..14b3c95db 100644 --- a/Website/Packages/datax-metrics/src/modules/metrics/apiConstants.js +++ b/Website/Packages/datax-metrics/src/modules/metrics/apiConstants.js @@ -8,5 +8,7 @@ export const Constants = { // List of supported services services: { flow: 'Flow.ManagementService' - } + }, + + batching: 'batching' }; diff --git a/Website/Packages/datax-metrics/src/modules/metrics/components/metricDashboard.jsx b/Website/Packages/datax-metrics/src/modules/metrics/components/metricDashboard.jsx index b833e7c84..8eb882789 100644 --- a/Website/Packages/datax-metrics/src/modules/metrics/components/metricDashboard.jsx +++ b/Website/Packages/datax-metrics/src/modules/metrics/components/metricDashboard.jsx @@ -10,6 +10,7 @@ import metricsDatasource from '../metrics.datasource'; import ServiceGroup from '../serviceGroup'; import * as Api from '../api'; import { LoadingPanel, StatementBox } from 'datax-common'; +import { Constants } from '../apiConstants'; class MetricDashboard extends React.Component { constructor() { @@ -34,7 +35,11 @@ class MetricDashboard extends React.Component { } componentDidMount() { - if (!this.props.product) { + if ( + !this.props.product || + !this.props.product.jobNames || + (this.props.product.jobNames && this.props.product.jobNames.length < 1) + ) { return; } @@ -59,6 +64,22 @@ class MetricDashboard extends React.Component { if (!product) { return null; + } else if (!product.jobNames || (product.jobNames && product.jobNames.length < 1)) { + const message = 'No metrics to display. No job for the flow is created. Please start the job to see metrics.'; + return ( +
+
{product.displayName} Metrics
+ +
+ ); + } else if (product.gui.input.mode === Constants.batching) { + const message = 'Metrics is not supported for the batch job.'; + return ( +
+
{product.displayName} Metrics
+ +
+ ); } else if (this.state.loading) { return ; } else if (!this.state.jobsAllRunning) { diff --git a/Website/Packages/datax-metrics/src/modules/metrics/components/metricExplorer.jsx b/Website/Packages/datax-metrics/src/modules/metrics/components/metricExplorer.jsx index 2e46fa3b6..44ef8a683 100644 --- a/Website/Packages/datax-metrics/src/modules/metrics/components/metricExplorer.jsx +++ b/Website/Packages/datax-metrics/src/modules/metrics/components/metricExplorer.jsx @@ -129,8 +129,7 @@ const contentStyle = { width: '100%', display: 'flex', flexDirection: 'column', - overflowY: 'auto', - paddingBottom: 80 + overflowY: 'auto' }; export default withRouter( diff --git a/Website/Packages/datax-pipeline/package.json b/Website/Packages/datax-pipeline/package.json index 3952e1b4e..5c4af2fc9 100644 --- a/Website/Packages/datax-pipeline/package.json +++ b/Website/Packages/datax-pipeline/package.json @@ -1,6 +1,6 @@ { "name": "datax-pipeline", - "version": "1.2.2", + "version": "1.3.0", "description": "Pipeline features", "author": "Microsoft", "license": "MIT", @@ -19,9 +19,12 @@ "test": "echo \"Error: no test specified\" && exit 1", "build": "rimraf dist && webpack --mode production --config webpack.config.js --bail --progress --display-modules --display-error-details --colors --display-chunks --display-reasons", "dev": "rimraf dist && webpack --mode development --config webpack.config.js --bail --progress --display-modules --display-error-details --colors --display-chunks --display-reasons", - "devpatch": "rimraf dist && webpack --mode development --config webpack.config.js --bail --progress --display-modules --display-error-details --colors --display-chunks --display-reasons && localdevpatch.bat" + "devpatch": "rimraf dist && webpack --mode development --config webpack.config.js --bail --progress --display-modules --display-error-details --colors --display-chunks --display-reasons && localdevpatch.bat", + "devwatch": "webpack --mode development --config webpack.config.js --bail --progress --display-modules --display-error-details --colors --display-chunks --display-reasons --watch" + }, + "dependencies": { + "react-datetime-picker": "2.4.0" }, - "dependencies": {}, "devDependencies": { "@babel/core": "7.1.6", "@babel/plugin-syntax-dynamic-import": "7.0.0", @@ -30,11 +33,9 @@ "babel-loader": "8.0.4", "css-loader": "1.0.1", "file-loader": "2.0.0", - "jsoneditor": "5.26.2", - "jsoneditor-react": "1.0.0", "mini-css-extract-plugin": "0.4.4", - "monaco-editor-webpack-plugin": "1.6.0", - "datax-common": "1.2.2", + "datax-common": "1.3.0", + "datax-query": "1.3.0", "office-ui-fabric-react": "6.111.2", "q": "1.5.1", "promise-polyfill": "8.1.0", @@ -43,7 +44,6 @@ "react": "16.6.3", "react-ace": "6.2.0", "react-dom": "16.6.3", - "react-monaco-editor": "0.22.0", "react-redux": "5.1.1", "react-router": "4.3.1", "react-router-dom": "4.3.1", diff --git a/Website/Packages/datax-pipeline/src/common/api.js b/Website/Packages/datax-pipeline/src/common/api.js index 8bd0bb06e..65384aac5 100644 --- a/Website/Packages/datax-pipeline/src/common/api.js +++ b/Website/Packages/datax-pipeline/src/common/api.js @@ -2,7 +2,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* -import { nodeServiceGetApi } from 'datax-common'; -import { ApiNames } from './apiConstants'; +import { nodeServiceGetApi, ApiNames } from 'datax-common'; export const functionEnabled = () => nodeServiceGetApi(ApiNames.FunctionEnabled); +export const isDatabricksSparkType = () => nodeServiceGetApi(ApiNames.IsDatabricksSparkType); diff --git a/Website/Packages/datax-pipeline/src/index.js b/Website/Packages/datax-pipeline/src/index.js index b12dc0f5e..d629d63e8 100644 --- a/Website/Packages/datax-pipeline/src/index.js +++ b/Website/Packages/datax-pipeline/src/index.js @@ -3,14 +3,12 @@ // Licensed under the MIT License // ********************************************************************* -import 'jsoneditor-react/es/editor.min.css'; -import 'monaco-editor/min/vs/editor/editor.main.css'; import './styles/styles.css'; +import 'datax-query/dist/css/index.css'; import flowslist from './modules/flowList/reducer'; import flow from './modules/flowDefinition/flowReducer'; -import flowKernel from './modules/flowDefinition/kernelReducer'; -import flowLayoutSettings from './modules/flowDefinition/layoutReducer'; +import { kernelReducer, layoutReducer, queryReducer } from 'datax-query'; /** * Website Consumption Contract @@ -46,8 +44,9 @@ import flowLayoutSettings from './modules/flowDefinition/layoutReducer'; export const reducers = { flowslist, flow, - flowKernel, - flowLayoutSettings + queryKernel: kernelReducer, + queryLayoutSettings: layoutReducer, + query: queryReducer }; // Exported React Components diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/api.js b/Website/Packages/datax-pipeline/src/modules/flowDefinition/api.js index 85b1ff6be..dda4bffb2 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/api.js +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/api.js @@ -3,16 +3,13 @@ // Licensed under the MIT License // ********************************************************************* import Q from 'q'; -import { serviceGetApi, servicePostApi, nodeServiceGetApi } from 'datax-common'; -import { Constants, ApiNames } from '../../common/apiConstants'; -import * as Models from './flowModels'; -import * as Helpers from './flowHelpers'; +import { serviceGetApi, servicePostApi, nodeServiceGetApi, Constants, ApiNames } from 'datax-common'; // Flow Service export const getFlow = name => getProduct(name).then(f => f.gui); -export const saveFlow = config => - servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.flow, 'flow/save', config); +export const saveFlow = (config, query) => + servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.flow, 'flow/save', config, query); export const deleteFlow = flow => servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.flow, 'flow/delete', { @@ -24,68 +21,7 @@ export const deleteFlow = flow => inputSubscriptionId: flow.input.properties.inputSubscriptionId, inputResourceGroup: flow.input.properties.inputResourceGroup, eventHubNames: flow.input.properties.inputEventhubName, - inputType: flow.input.type, - }); - -export const getTableSchemas = flow => - servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.flow, 'userqueries/schema', { - name: flow.name, - displayName: flow.displayName, - query: flow.query, - inputSchema: flow.input.properties.inputSchemaFile, - rules: Helpers.convertFlowToConfigRules(flow.rules), - outputTemplates: flow.outputTemplates - }); - -export const getCodeGenQuery = flow => - servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.flow, 'userqueries/codegen', { - name: flow.name, - displayName: flow.displayName, - query: flow.query, - rules: Helpers.convertFlowToConfigRules(flow.rules), - outputTemplates: flow.outputTemplates - }); - -// Interactive Query -export const getDiagnosticKernel = flow => - servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.interactiveQuery, 'kernel', { - name: flow.name, - displayName: flow.displayName, - userName: flow.owner, - inputSchema: flow.input.properties.inputSchemaFile, - normalizationSnippet: flow.input.properties.normalizationSnippet, - refData: flow.referenceData, - functions: flow.functions - }); - -export const refreshDiagnosticKernel = (flow, kernelId) => - servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.interactiveQuery, 'kernel/refresh', { - kernelId: kernelId, - userName: flow.owner, - name: flow.name, - displayName: flow.displayName, - inputSchema: flow.input.properties.inputSchemaFile, - normalizationSnippet: flow.input.properties.normalizationSnippet, - refData: flow.referenceData, - functions: flow.functions - }); - -export const deleteAllKernels = () => - servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.interactiveQuery, 'kernels/deleteall', {}); - -export const deleteDiagnosticKernelOnUnload = kernelId => - servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.interactiveQuery, 'kernel/delete', kernelId); - -export const deleteDiagnosticKernel = deleteDiagnosticKernelOnUnload; - -export const executeQuery = (flow, selectedQuery, kernelId) => - servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.interactiveQuery, 'kernel/executequery', { - name: flow.name, - displayName: flow.displayName, - query: selectedQuery, - kernelId: kernelId, - rules: Helpers.convertFlowToConfigRules(flow.rules), - outputTemplates: flow.outputTemplates + inputType: flow.input.type }); // Schema Inference @@ -99,32 +35,11 @@ export const getInputSchema = flow => inputResourceGroup: flow.input.properties.inputResourceGroup, eventHubNames: flow.input.properties.inputEventhubName, inputType: flow.input.type, + inputMode: flow.input.mode, + batchInputs: flow.batchInputs, seconds: flow.samplingInputDuration }); -// Live Data -export const resampleInput = (flow, kernelId) => - servicePostApi( - Constants.serviceRouteApi, - Constants.serviceApplication, - Constants.services.liveData, - 'inputdata/refreshsampleandkernel', - { - name: flow.name, - displayName: flow.displayName, - userName: flow.owner, - kernelId: kernelId, - inputSchema: flow.input.properties.inputSchemaFile, - normalizationSnippet: flow.input.properties.normalizationSnippet, - eventhubConnectionString: flow.input.properties.inputEventhubConnection, - inputSubscriptionId: flow.input.properties.inputSubscriptionId, - inputResourceGroup: flow.input.properties.inputResourceGroup, - eventHubNames: flow.input.properties.inputEventhubName, - inputType: flow.input.type, - seconds: flow.resamplingInputDuration - } - ); - // Product and Jobs export const getProduct = name => diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/flowDefinitionPanel.jsx b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/flowDefinitionPanel.jsx index af3147069..74b04f3f9 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/flowDefinitionPanel.jsx +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/flowDefinitionPanel.jsx @@ -12,10 +12,6 @@ import * as Helpers from '../flowHelpers'; import * as Models from '../flowModels'; import * as Actions from '../flowActions'; import * as Selectors from '../flowSelectors'; -import * as LayoutActions from '../layoutActions'; -import * as LayoutSelectors from '../layoutSelectors'; -import * as KernelActions from '../kernelActions'; -import * as KernelSelectors from '../kernelSelectors'; import { DefaultButton, PrimaryButton, Spinner, SpinnerSize, MessageBar, MessageBarType } from 'office-ui-fabric-react'; import { Dialog, DialogType, DialogFooter } from 'office-ui-fabric-react/lib/Dialog'; import { Modal } from 'office-ui-fabric-react/lib/Modal'; @@ -23,9 +19,9 @@ import InfoSettingsContent from './info/infoSettingsContent'; import InputSettingsContent from './input/inputSettingsContent'; import ReferenceDataSettingsContent from './referenceData/referenceDataSettingsContent'; import FunctionSettingsContent from './function/functionSettingsContent'; -import QuerySettingsContent from './query/querySettingsContent'; import ScaleSettingsContent from './scale/scaleSettingsContent'; import OutputSettingsContent from './output/outputSettingsContent'; +import ScheduleSettingsContent from './schedule/ScheduleSettingsContent'; import RulesSettingsContent from './rule/rulesSettingsContent'; import { functionEnabled } from '../../../common/api'; import { @@ -39,6 +35,16 @@ import { VerticalTabItem, getApiErrorMessage } from 'datax-common'; +import { + QueryApi, + KernelActions, + KernelSelectors, + QueryActions, + LayoutActions, + LayoutSelectors, + QuerySelectors, + QuerySettingsContent +} from 'datax-query'; class FlowDefinitionPanel extends React.Component { constructor(props) { @@ -59,6 +65,7 @@ class FlowDefinitionPanel extends React.Component { inProgessDialogMessage: '', havePermission: true, saveFlowButtonEnabled: false, + deployFlowButtonEnabled: false, deleteFlowButtonEnabled: false, getInputSchemaButtonEnabled: false, outputSideToolBarEnabled: false, @@ -88,7 +95,10 @@ class FlowDefinitionPanel extends React.Component { addOutputSinkButtonEnabled: false, deleteOutputSinkButtonEnabled: false, scaleNumExecutorsSliderEnabled: false, - scaleExecutorMemorySliderEnabled: false + scaleExecutorMemorySliderEnabled: false, + + addBatchButtonEnabled: false, + deleteBatchButtonEnabled: false }; this.handleWindowClose = this.handleWindowClose.bind(this); @@ -101,7 +111,7 @@ class FlowDefinitionPanel extends React.Component { } handleWindowClose() { - Api.deleteDiagnosticKernelOnUnload(this.props.kernelId); + QueryApi.deleteDiagnosticKernelOnUnload(this.props.kernelId); this.sleep(500); } @@ -110,7 +120,7 @@ class FlowDefinitionPanel extends React.Component { clearInterval(this.state.timer); if (this.props.kernelId !== undefined && this.props.kernelId !== '') { - this.onDeleteKernel(this.props.kernelId); + this.onDeleteKernel(this.props.kernelId, this.props.flow.name); } } @@ -152,6 +162,7 @@ class FlowDefinitionPanel extends React.Component { functionEnabled().then(response => { this.setState({ saveFlowButtonEnabled: response.saveFlowButtonEnabled ? true : false, + deployFlowButtonEnabled: response.deployFlowButtonEnabled ? true : false, deleteFlowButtonEnabled: response.deleteFlowButtonEnabled ? true : false, getInputSchemaButtonEnabled: response.getInputSchemaButtonEnabled ? true : false, outputSideToolBarEnabled: response.outputSideToolBarEnabled ? true : false, @@ -181,7 +192,10 @@ class FlowDefinitionPanel extends React.Component { addOutputSinkButtonEnabled: response.addOutputSinkButtonEnabled ? true : false, deleteOutputSinkButtonEnabled: response.deleteOutputSinkButtonEnabled ? true : false, scaleNumExecutorsSliderEnabled: response.scaleNumExecutorsSliderEnabled ? true : false, - scaleExecutorMemorySliderEnabled: response.scaleExecutorMemorySliderEnabled ? true : false + scaleExecutorMemorySliderEnabled: response.scaleExecutorMemorySliderEnabled ? true : false, + + addBatchButtonEnabled: response.addBatchButtonEnabled ? true : false, + deleteBatchButtonEnabled: response.deleteBatchButtonEnabled ? true : false }); }); } @@ -203,13 +217,19 @@ class FlowDefinitionPanel extends React.Component { renderButtons() { if (this.state.havePermission) { - return [this.renderBackButton(), this.renderSaveButton(), this.renderDeleteButton(), this.renderDebugButton()]; + return [ + this.renderBackButton(), + this.renderSaveButton(), + this.renderDeployButton(), + this.renderDeleteButton(), + this.renderDebugButton() + ]; } } renderBackButton() { - const buttonTooltip = this.props.flow.isDirty ? 'Discard Changes' : 'Go Back'; - let buttonText = this.props.flow.isDirty ? 'Cancel' : 'Back'; + const buttonTooltip = this.props.flow.isDirty || this.props.isQueryDirty ? 'Discard Changes' : 'Go Back'; + let buttonText = this.props.flow.isDirty || this.props.isQueryDirty ? 'Cancel' : 'Back'; let buttonIcon = ; if (this.state.loading) { @@ -228,19 +248,45 @@ class FlowDefinitionPanel extends React.Component { renderSaveButton() { if (!this.state.loading) { - const enableButton = this.props.flow.isDirty && this.props.flowValidated && this.state.saveFlowButtonEnabled; + //Will be enabled when user has access and flow has been changed. + const enableButton = (this.props.flow.isDirty || this.props.isQueryDirty) && this.state.saveFlowButtonEnabled; return ( this.onSaveDefinition()} > + Save + + ); + } else { + return null; + } + } + + renderDeployButton() { + if (!this.state.loading) { + const enableButton = + //Will be enabled when user has access and flow has valid configuration. + this.props.flowValidated && this.state.deployFlowButtonEnabled; + return ( + this.onDeployDefinition()} + > + Deploy ); @@ -313,6 +359,35 @@ class FlowDefinitionPanel extends React.Component { ); } + renderScheduleTab() { + if (this.props.input.mode === Models.inputModeEnum.streaming) { + return null; + } else { + return ( + + + + ); + } + } + renderFlow() { return (
@@ -323,14 +398,21 @@ class FlowDefinitionPanel extends React.Component { displayName={this.props.flow.displayName} name={this.props.flow.name} owner={this.props.flow.owner} + databricksToken={this.props.flow.databricksToken} onUpdateDisplayName={this.props.onUpdateDisplayName} + onUpdateDatabricksToken={this.props.onUpdateDatabricksToken} flowNameTextboxEnabled={this.state.flowNameTextboxEnabled} + isDatabricksSparkType={this.props.flow.isDatabricksSparkType} + saveFlowButtonEnabled={this.state.saveFlowButtonEnabled} + saveFlowAndInitializeKernel={() => this.saveFlowAndInitializeKernel()} /> this.onGetCodeGenQuery()} onRefreshKernel={kernelId => this.refreshKernel(kernelId)} - onDeleteAllKernels={this.props.onDeleteAllKernels} + onDeleteAllKernels={() => this.deleteAllKernel()} onExecuteQuery={(selectedQuery, kernelId) => this.onExecuteQuery(selectedQuery, kernelId)} onResampleInput={kernelId => this.onResampleInput(kernelId)} onShowTestQueryOutputPanel={isVisible => this.onShowTestQueryOutputPanel(isVisible)} @@ -477,6 +568,10 @@ class FlowDefinitionPanel extends React.Component { onUpdateBlobPartitionFormat={this.props.onUpdateBlobPartitionFormat} onUpdateFormatType={this.props.onUpdateFormatType} onUpdateCompressionType={this.props.onUpdateCompressionType} + onUpdateSqlConnection={this.props.onUpdateSqlConnection} + onUpdateSqlTableName={this.props.onUpdateSqlTableName} + onUpdateSqlWriteMode={this.props.onUpdateSqlWriteMode} + onUpdateSqlUseBulkInsert={this.props.onUpdateSqlUseBulkInsert} addOutputSinkButtonEnabled={this.state.addOutputSinkButtonEnabled} deleteOutputSinkButtonEnabled={this.state.deleteOutputSinkButtonEnabled} /> @@ -489,8 +584,14 @@ class FlowDefinitionPanel extends React.Component { onUpdateExecutorMemory={this.props.onUpdateExecutorMemory} scaleNumExecutorsSliderEnabled={this.state.scaleNumExecutorsSliderEnabled} scaleExecutorMemorySliderEnabled={this.state.scaleExecutorMemorySliderEnabled} + isDatabricksSparkType={this.props.flow.isDatabricksSparkType} + onUpdateDatabricksAutoScale={this.props.onUpdateDatabricksAutoScale} + onUpdateDatabricksMinWorkers={this.props.onUpdateDatabricksMinWorkers} + onUpdateDatabricksMaxWorkers={this.props.onUpdateDatabricksMaxWorkers} /> + + {this.renderScheduleTab()}
@@ -573,6 +674,39 @@ class FlowDefinitionPanel extends React.Component { } onSaveDefinition() { + this.setState({ + isSaving: true, + showMessageBar: false, + messageBarIsError: false, + inProgessDialogMessage: 'Save in progress' + }); + + this.props + .onSaveFlow(this.props.flow, this.props.query) + .then(name => { + this.setState({ + isSaving: false, + showMessageBar: true, + messageBarIsError: false, + messageBarStatement: + this.props.input.mode === Models.inputModeEnum.streaming + ? 'Flow definition is saved' + : 'Flow definition is saved. If there are any scheduled batch jobs in this flow, they will be created and started by the scheduler.' + }); + + this.props.initFlow({ id: name }); + }) + .catch(error => { + this.setState({ + isSaving: false, + showMessageBar: true, + messageBarIsError: true, + messageBarStatement: getApiErrorMessage(error) + }); + }); + } + + onDeployDefinition() { this.setState({ isSaving: true, showMessageBar: false, @@ -581,7 +715,7 @@ class FlowDefinitionPanel extends React.Component { }); this.props - .onSaveFlow(this.props.flow) + .onDeployFlow(this.props.flow, this.props.query) .then(name => { this.setState({ isSaving: false, @@ -654,7 +788,7 @@ class FlowDefinitionPanel extends React.Component { console.log(this.props.flow); console.log('Config object'); - const config = Helpers.convertFlowToConfig(this.props.flow); + const config = Helpers.convertFlowToConfig(this.props.flow, this.props.query); console.log(config); console.log('Flow object from Config'); @@ -688,11 +822,11 @@ class FlowDefinitionPanel extends React.Component { } onGetTableSchemas() { - return this.props.onGetTableSchemas(this.props.flow); + return this.props.onGetTableSchemas(Helpers.convertFlowToQueryMetadata(this.props.flow, this.props.query)); } onGetCodeGenQuery() { - return this.props.onGetCodeGenQuery(this.props.flow); + return this.props.onGetCodeGenQuery(Helpers.convertFlowToQueryMetadata(this.props.flow, this.props.query)); } getKernel() { @@ -702,30 +836,51 @@ class FlowDefinitionPanel extends React.Component { ) { const version = this.props.kernelVersion + 1; this.props.onUpdateKernelVersion(version); - this.props.onGetKernel(this.props.flow, version, Actions.updateErrorMessage); + this.props.onGetKernel( + Helpers.convertFlowToQueryMetadata(this.props.flow, this.props.query), + version, + QueryActions.updateErrorMessage + ); } } refreshKernel(kernelId) { const version = this.props.kernelVersion + 1; this.props.onUpdateKernelVersion(version); - this.props.onRefreshKernel(this.props.flow, kernelId, version, Actions.updateErrorMessage); + this.props.onRefreshKernel( + Helpers.convertFlowToQueryMetadata(this.props.flow, this.props.query), + kernelId, + version, + QueryActions.updateErrorMessage + ); + } + + deleteAllKernel() { + this.props.onDeleteAllKernels(QueryActions.updateErrorMessage, this.props.flow.name); } - onDeleteKernel(kernelId) { + onDeleteKernel(kernelId, flowName) { const version = this.props.kernelVersion + 1; this.props.onUpdateKernelVersion(version); - return this.props.onDeleteKernel(kernelId, version); + return this.props.onDeleteKernel(kernelId, version, flowName); } onResampleInput(kernelId) { const version = this.props.kernelVersion + 1; this.props.onUpdateKernelVersion(version); - this.props.onResampleInput(this.props.flow, kernelId, version); + this.props.onResampleInput(Helpers.convertFlowToQueryMetadata(this.props.flow, this.props.query), kernelId, version); } onExecuteQuery(selectedQuery, kernelId) { - return this.props.onExecuteQuery(this.props.flow, selectedQuery, kernelId); + return this.props.onExecuteQuery(Helpers.convertFlowToQueryMetadata(this.props.flow, this.props.query), selectedQuery, kernelId); + } + + saveFlowAndInitializeKernel() { + Q( + this.onSaveDefinition() + ).then(() => { + this.getKernel(); + }) } } @@ -741,13 +896,16 @@ const mapStateToProps = state => ({ input: Selectors.getFlowInput(state), referenceData: Selectors.getFlowReferenceData(state), functions: Selectors.getFlowFunctions(state), - query: Selectors.getFlowQuery(state), + query: QuerySelectors.getQueryContent(state), + isQueryDirty: QuerySelectors.getQueryDirty(state), scale: Selectors.getFlowScale(state), outputs: Selectors.getFlowOutputs(state), outputTemplates: Selectors.getFlowOutputTemplates(state), rules: Selectors.getFlowRules(state), // State + selectedBatchIndex: Selectors.getSelectedBatchIndex(state), + selectedReferenceDataIndex: Selectors.getSelectedReferenceDataIndex(state), selectedFunctionIndex: Selectors.getSelectedFunctionIndex(state), selectedSinkerIndex: Selectors.getSelectedSinkerIndex(state), @@ -766,31 +924,39 @@ const mapStateToProps = state => ({ inputValidated: Selectors.validateFlowInput(state), referenceDataValidated: Selectors.validateFlowReferenceData(state), functionsValidated: Selectors.validateFlowFunctions(state), - queryValidated: Selectors.validateFlowQuery(state), scaleValidated: Selectors.validateFlowScale(state), outputsValidated: Selectors.validateFlowOutputs(state), rulesValidated: Selectors.validateFlowRules(state), - flowValidated: Selectors.validateFlow(state) + flowValidated: Selectors.validateFlow(state), + queryValidated: QuerySelectors.validateQueryTab(state), + scheduleValidated: Selectors.validateFlowSchedule(state) }); // Dispatch Props const mapDispatchToProps = dispatch => ({ // Init Actions initFlow: context => dispatch(Actions.initFlow(context)), - // Message Actions onUpdateWarningMessage: message => dispatch(Actions.updateWarningMessage(message)), // Info Actions onUpdateDisplayName: displayName => dispatch(Actions.updateDisplayName(displayName)), + onUpdateDatabricksToken: databricksToken => dispatch(Actions.updateDatabricksToken(databricksToken)), // Input Actions onUpdateInputMode: mode => dispatch(Actions.updateInputMode(mode)), onUpdateInputType: type => dispatch(Actions.updateInputType(type)), onUpdateInputHubName: name => dispatch(Actions.updateInputHubName(name)), + onUpdateInputPath: path => dispatch(Actions.updateInputPath(path)), onUpdateInputHubConnection: connection => dispatch(Actions.updateInputHubConnection(connection)), onUpdateInputSubscriptionId: id => dispatch(Actions.updateInputSubscriptionId(id)), onUpdateInputResourceGroup: name => dispatch(Actions.updateInputResourceGroup(name)), + + onUpdateBatchInputConnection: connection => dispatch(Actions.updateBatchInputConnection(connection)), + onUpdateBatchInputPath: path => dispatch(Actions.updateBlobInputPath(path)), + onUpdateBatchInputFormatType: formatType => dispatch(Actions.updateBatchInputFormatType(formatType)), + onUpdateBatchInputCompressionType: compressionType => dispatch(Actions.updateBachInputCompressionType(compressionType)), + onUpdateInputWindowDuration: duration => dispatch(Actions.updateInputWindowDuration(duration)), onUpdateInputTimestampColumn: duration => dispatch(Actions.updateInputTimestampColumn(duration)), onUpdateInputWatermarkValue: duration => dispatch(Actions.updateInputWatermarkValue(duration)), @@ -827,17 +993,18 @@ const mapDispatchToProps = dispatch => ({ onUpdateAzureFunctionParams: params => dispatch(Actions.updateAzureFunctionParams(params)), // Query Actions - onUpdateQuery: query => dispatch(Actions.updateQuery(query)), - onGetCodeGenQuery: flow => Actions.getCodeGenQuery(flow), - onExecuteQuery: (flow, selectedQuery, kernelId) => dispatch(Actions.executeQuery(flow, selectedQuery, kernelId)), - onGetKernel: (flow, version, updateErrorMessage) => dispatch(KernelActions.getKernel(flow, version, updateErrorMessage)), + onUpdateQuery: query => dispatch(QueryActions.updateQuery(query)), + onGetCodeGenQuery: queryMetadata => QueryActions.getCodeGenQuery(queryMetadata), + onExecuteQuery: (queryMetadata, selectedQuery, kernelId) => dispatch(QueryActions.executeQuery(queryMetadata, selectedQuery, kernelId)), + onGetKernel: (queryMetadata, version, updateErrorMessage) => + dispatch(KernelActions.getKernel(queryMetadata, version, updateErrorMessage)), onUpdateKernelVersion: version => dispatch(KernelActions.updateKernelVersion(version)), - onRefreshKernel: (flow, kernelId, version, updateErrorMessage) => - dispatch(KernelActions.refreshKernel(flow, kernelId, version, updateErrorMessage)), - onDeleteKernel: (kernelId, version) => dispatch(KernelActions.deleteKernel(kernelId, version)), - onResampleInput: (flow, kernelId, version) => dispatch(Actions.resampleInput(flow, kernelId, version)), - onUpdateResamplingInputDuration: duration => dispatch(Actions.updateResamplingInputDuration(duration)), - onDeleteAllKernels: updateErrorMessage => dispatch(KernelActions.deleteAllKernels(updateErrorMessage)), + onRefreshKernel: (queryMetadata, kernelId, version, updateErrorMessage) => + dispatch(KernelActions.refreshKernel(queryMetadata, kernelId, version, updateErrorMessage)), + onDeleteKernel: (kernelId, version, flowName) => dispatch(KernelActions.deleteKernel(kernelId, version, flowName)), + onResampleInput: (queryMetadata, kernelId, version) => dispatch(QueryActions.resampleInput(queryMetadata, kernelId, version)), + onUpdateResamplingInputDuration: duration => dispatch(QueryActions.updateResamplingInputDuration(duration)), + onDeleteAllKernels: (updateErrorMessage, flowName) => dispatch(KernelActions.deleteAllKernels(updateErrorMessage, flowName)), // Query Pane Layout Actions onShowTestQueryOutputPanel: isVisible => dispatch(LayoutActions.onShowTestQueryOutputPanel(isVisible)), @@ -845,6 +1012,9 @@ const mapDispatchToProps = dispatch => ({ // Scale Actions onUpdateNumExecutors: numExecutors => dispatch(Actions.updateNumExecutors(numExecutors)), onUpdateExecutorMemory: executorMemory => dispatch(Actions.updateExecutorMemory(executorMemory)), + onUpdateDatabricksAutoScale: databricksAutoScale => dispatch(Actions.updateDatabricksAutoScale(databricksAutoScale)), + onUpdateDatabricksMinWorkers: databricksMinWorkers => dispatch(Actions.updateDatabricksMinWorkers(databricksMinWorkers)), + onUpdateDatabricksMaxWorkers: databricksMaxWorkers => dispatch(Actions.updateDatabricksMaxWorkers(databricksMaxWorkers)), // Output Actions onNewSinker: type => dispatch(Actions.newSinker(type)), @@ -862,6 +1032,11 @@ const mapDispatchToProps = dispatch => ({ onUpdateFormatType: type => dispatch(Actions.updateFormatType(type)), onUpdateCompressionType: type => dispatch(Actions.updateCompressionType(type)), + onUpdateSqlConnection: connection => dispatch(Actions.updateSqlConnection(connection)), + onUpdateSqlTableName: name => dispatch(Actions.updateSqlTableName(name)), + onUpdateSqlWriteMode: mode => dispatch(Actions.updateSqlWriteMode(mode)), + onUpdateSqlUseBulkInsert: useBulkInsert => dispatch(Actions.updateSqlUseBulkInsert(useBulkInsert)), + // Rule Actions onNewRule: type => dispatch(Actions.newRule(type)), onDeleteRule: index => dispatch(Actions.deleteRule(index)), @@ -877,14 +1052,29 @@ const mapDispatchToProps = dispatch => ({ onUpdateTagAggregates: aggregates => dispatch(Actions.updateTagAggregates(aggregates)), onUpdateTagPivots: pivots => dispatch(Actions.updateTagPivots(pivots)), onUpdateSchemaTableName: name => dispatch(Actions.updateSchemaTableName(name)), - onGetTableSchemas: flow => Actions.getTableSchemas(flow), + onGetTableSchemas: queryMetadata => QueryActions.getTableSchemas(queryMetadata), // Save and Delete Actions - onSaveFlow: flow => Actions.saveFlow(flow), + onSaveFlow: (flow, query) => Actions.saveFlow(flow, query), + onDeployFlow: (flow, query) => Actions.deployFlow(flow, query), onDeleteFlow: flow => Actions.deleteFlow(flow), // enableOneBox Action - onUpdateOneBoxMode: enableOneBox => dispatch(Actions.updateOneBoxMode(enableOneBox)) + onUpdateOneBoxMode: enableOneBox => dispatch(Actions.updateOneBoxMode(enableOneBox)), + + // Schedule Actions + onNewBatch: type => dispatch(Actions.newBatch(type)), + onDeleteBatch: index => dispatch(Actions.deleteBatch(index)), + onUpdateSelectedBatchIndex: index => dispatch(Actions.updateSelectedBatchIndex(index)), + onUpdateBatchName: value => dispatch(Actions.updateBatchName(value)), + onUpdateBatchStartTime: value => dispatch(Actions.updateBatchStartTime(value)), + onUpdateBatchEndTime: value => dispatch(Actions.updateBatchEndTime(value)), + onUpdateBatchIntervalValue: value => dispatch(Actions.updateBatchIntervalValue(value)), + onUpdateBatchIntervalType: value => dispatch(Actions.updateBatchIntervalType(value)), + onUpdateBatchDelayValue: value => dispatch(Actions.updateBatchDelayValue(value)), + onUpdateBatchDelayType: value => dispatch(Actions.updateBatchDelayType(value)), + onUpdateBatchWindowValue: value => dispatch(Actions.updateBatchWindowValue(value)), + onUpdateBatchWindowType: value => dispatch(Actions.updateBatchWindowType(value)) }); // Styles diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/function/functionSettingsContent.jsx b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/function/functionSettingsContent.jsx index 9b6520776..959f96da6 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/function/functionSettingsContent.jsx +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/function/functionSettingsContent.jsx @@ -113,7 +113,7 @@ export default class FunctionSettingsContent extends React.Component { renderAddFunctionButton() { const menuItems = this.props.enableLocalOneBox ? Models.functionTypes - .filter(functionType => functionType.name != 'Azure Function') // disable Azure function in OneBox mode + .filter(functionType => functionType.name !== 'Azure Function') // disable Azure function in OneBox mode .map(functionType => { return Object.assign({}, functionType, { onClick: () => this.props.onNewFunction(functionType.key) diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/info/infoSettingsContent.jsx b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/info/infoSettingsContent.jsx index 08ea4ecbe..ed87e3409 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/info/infoSettingsContent.jsx +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/info/infoSettingsContent.jsx @@ -4,8 +4,8 @@ // ********************************************************************* import React from 'react'; import PropTypes from 'prop-types'; -import { TextField } from 'office-ui-fabric-react'; -import { Colors, ScrollableContentPane, StatementBox } from 'datax-common'; +import { DefaultButton, TextField } from 'office-ui-fabric-react'; +import { Colors, IconButtonStyles, ScrollableContentPane, StatementBox } from 'datax-common'; export default class InfoSettingsContent extends React.Component { constructor(props) { @@ -35,6 +35,24 @@ export default class InfoSettingsContent extends React.Component { /> + {this.props.isDatabricksSparkType && ( +
+ this.props.onUpdateDatabricksToken(databricksToken)} + /> +
+ {this.renderSaveButton()} +
+
+ )} +
); } + + renderSaveButton() { + //Will be enabled when user has access and the token is not empty + const enableButton = this.props.saveFlowButtonEnabled && this.props.databricksToken; + return ( + this.props.saveFlowAndInitializeKernel()} + > + + Save + + ); + } } // Props @@ -66,8 +104,8 @@ InfoSettingsContent.propTypes = { displayName: PropTypes.string.isRequired, name: PropTypes.string.isRequired, owner: PropTypes.string.isRequired, - onUpdateDisplayName: PropTypes.func.isRequired, + onUpdateDatabricksToken: PropTypes.func.isRequired, flowNameTextboxEnabled: PropTypes.bool.isRequired }; @@ -87,3 +125,14 @@ const contentStyle = { const sectionStyle = { paddingBottom: 15 }; + +const rowFlexSettingsStyle = { + display: 'flex', + flexDirection: 'row', + paddingBottom: 15 +}; + +const rightSideButtonStyle = { + paddingLeft: 5, + paddingTop: 28 +}; \ No newline at end of file diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/input/inputSettingsContent.jsx b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/input/inputSettingsContent.jsx index de99b8f2b..d8b41561d 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/input/inputSettingsContent.jsx +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/input/inputSettingsContent.jsx @@ -7,15 +7,23 @@ import PropTypes from 'prop-types'; import * as Helpers from '../../flowHelpers'; import * as Models from '../../flowModels'; import { Label, TextField, Toggle, Dropdown, DefaultButton } from 'office-ui-fabric-react'; -import { JsonEditor } from 'jsoneditor-react'; -import 'jsoneditor-react/es/editor.min.css'; import ace from 'brace'; import 'brace/mode/json'; import 'brace/theme/textmate'; -import MonacoEditor from 'react-monaco-editor'; +import 'datax-query/dist/css/index.css'; import 'brace/mode/sql'; import 'brace/theme/xcode'; -import { Colors, IconButtonStyles, ScrollableContentPane, StatementBox, LoadingPanel, getApiErrorMessage } from 'datax-common'; +import { + Colors, + IconButtonStyles, + ScrollableContentPane, + StatementBox, + LoadingPanel, + getApiErrorMessage, + CommonHelpers +} from 'datax-common'; +import { JsonEditor } from 'datax-query'; +import { MonacoEditorControl } from 'datax-query'; const inputSchemaExampleWiki = 'https://aka.ms/data-accelerator-input'; const normalizationExampleWiki = 'https://aka.ms/data-accelerator-normalization'; @@ -26,6 +34,7 @@ export default class InputSettingsContent extends React.Component { this.state = { showNormalizationSnippet: false, + isBackfilling: false, error: {} }; } @@ -44,7 +53,7 @@ export default class InputSettingsContent extends React.Component { renderContent() { return ( -
+
{this.renderLeftPane()} {this.renderRightPane()} @@ -54,53 +63,78 @@ export default class InputSettingsContent extends React.Component { } renderLeftPane() { - return ( -
- -
- {this.renderModeDropdown()} - {this.renderTypeDropdown()} - {this.renderEventHubName()} - {this.renderEventHubConnection()} - {this.renderSubscriptionId()} - {this.renderResourceGroup()} -
- -
- -
-
- this.props.onUpdateWindowDuration(value)} - onGetErrorMessage={value => this.validateNumber(value)} - disabled={!this.props.inputWindowDurationTextboxEnabled} - /> + if (this.props.input.mode === Models.inputModeEnum.batching) { + let batchData = undefined; + if (this.props.batchInputs && this.props.selectedFlowBatchInputIndex !== undefined) { + batchData = this.props.batchInputs[this.props.selectedFlowBatchInputIndex]; + } + return ( +
+ +
+ {this.renderModeDropdown()} + {this.renderTypeDropdown()} + {this.renderBlobInputConnection(batchData)} + {this.renderBlobInputPath(batchData)} + {this.renderInputFormatTypeDropdown(batchData)} + {this.renderInputCompressionTypeDropdown(batchData)}
-
- this.props.onUpdateMaxRate(value)} - onGetErrorMessage={value => this.validateNumber(value)} - disabled={!this.props.inputMaxRateTextboxEnabled} - /> +
+ +
{this.renderShowNormalizationSnippetToggle()}
+ +
+ ); + } else { + return ( +
+ +
+ {this.renderModeDropdown()} + {this.renderTypeDropdown()} + {this.renderEventHubName()} + {this.renderEventHubConnection()} + {this.renderSubscriptionId()} + {this.renderResourceGroup()}
- {this.renderTimestampColumn()} - {this.renderWatermark()} -
+
+ +
+
+ this.props.onUpdateWindowDuration(value)} + onGetErrorMessage={value => this.validateNumber(value)} + disabled={!this.props.inputWindowDurationTextboxEnabled} + /> +
+ +
+ this.props.onUpdateMaxRate(value)} + onGetErrorMessage={value => this.validateNumber(value)} + disabled={!this.props.inputMaxRateTextboxEnabled} + /> +
+ + {this.renderTimestampColumn()} + {this.renderWatermark()} +
-
+
-
{this.renderShowNormalizationSnippetToggle()}
- -
- ); +
{this.renderShowNormalizationSnippetToggle()}
+ +
+ ); + } } renderRightPane() { @@ -138,23 +172,31 @@ export default class InputSettingsContent extends React.Component { renderTypeDropdown() { const options = this.props.enableLocalOneBox ? Models.inputTypes - .filter(type => type.name === 'Local') - .map(type => { - return { - key: type.key, - text: type.name, - disabled: type.disabled - }; - }) - : Models.inputTypes - .filter(type => type.name !== 'Local') - .map(type => { - return { - key: type.key, - text: type.name, - disabled: type.disabled - }; - }); + .filter(type => type.name === 'Local') + .map(type => { + return { + key: type.key, + text: type.name, + disabled: type.disabled + }; + }) + : this.props.input.mode === Models.inputModeEnum.streaming + ? Models.inputTypes + .filter(type => type.name !== 'Local') + .map(type => { + return { + key: type.key, + text: type.name, + disabled: type.disabled + }; + }) + : Models.inputTypesBatching.map(type => { + return { + key: type.key, + text: type.name, + disabled: type.disabled + }; + }); return (
@@ -172,7 +214,10 @@ export default class InputSettingsContent extends React.Component { renderEventHubName() { if (this.props.input.type !== Models.inputTypeEnum.events && this.props.input.type !== Models.inputTypeEnum.local) { - const label = this.props.input.type === Models.inputTypeEnum.iothub ? 'Event Hub-Compatible Name' : 'Topics (Use commas to separate topics)'; + const label = + this.props.input.type === Models.inputTypeEnum.iothub + ? 'Event Hub-Compatible Name' + : 'Topics (Use commas to separate topics)'; return (
+ this.props.onUpdateBatchInputConnection(value)} + autoAdjustHeight + resizable={false} + disabled={!this.props.inputEventHubConnectionStringEnabled} + /> +
+ ); + } + } + + renderBlobInputPath(batchData) { + if (this.props.input.type !== Models.inputTypeEnum.blob) { return null; } else { + let value = batchData ? batchData.properties.path : undefined; + return ( +
+ this.props.onUpdateBatchInputPath(value)} + autoAdjustHeight + resizable={false} + /> +
+ ); + } + } + + renderInputFormatTypeDropdown(batchData) { + if (this.props.input.type !== Models.inputTypeEnum.blob) { + return null; + } else { + const options = Models.inputFormatTypes.map(type => { + return { + key: type.key, + text: type.name, + disabled: type.disabled + }; + }); + + let value = batchData ? batchData.properties.formatType : Models.inputFormatTypeEnum.json; + + return ( +
+ + this.props.onUpdateBatchInputFormatType(selection.key)} + /> +
+ ); + } + } + + renderInputCompressionTypeDropdown(batchData) { + if (this.props.input.type !== Models.inputTypeEnum.blob) { + return null; + } else { + const options = Models.inputCompressionTypes.map(type => { + return { + key: type.key, + text: type.name, + disabled: type.disabled + }; + }); + + let value = batchData ? batchData.properties.compressionType : Models.inputCompressionTypes.none; + + return ( +
+ + this.props.onUpdateBatchInputCompressionType(selection.key)} + /> +
+ ); + } + } + + renderSubscriptionId() { + if (this.props.input.type === Models.inputTypeEnum.events || this.props.input.type === Models.inputTypeEnum.iothub) { return (
); + } else { + return null; } } renderResourceGroup() { - if (this.props.input.type === Models.inputTypeEnum.local || this.props.input.type === Models.inputTypeEnum.kafka || this.props.input.type === Models.inputTypeEnum.kafkaeventhub) { - return null; - } else { + if (this.props.input.type === Models.inputTypeEnum.events || this.props.input.type === Models.inputTypeEnum.iothub) { return (
); + } else { + return null; } } @@ -344,13 +494,49 @@ export default class InputSettingsContent extends React.Component { ); } + renderSamplingInputLabel(mode) { + if (mode === Models.inputModeEnum.streaming) { + return ( +
+
+ +
+
+ this.props.onUpdateSamplingInputDuration(value)} + onGetErrorMessage={value => this.validateNumber(value)} + /> +
+
+ ); + } else { + return ( +
+
+ +
+
+ ); + } + } + renderGetInputSchemaButton() { const display = 'Get Schema'; const enableButton = - this.props.input.properties.inputEventhubConnection !== '' && + ((this.props.input.mode === Models.inputModeEnum.streaming && this.props.input.properties.inputEventhubConnection !== '') || + (this.props.input.mode === Models.inputModeEnum.batching && + this.props.batchInputs && + this.props.selectedFlowBatchInputIndex !== undefined && + this.props.batchInputs[this.props.selectedFlowBatchInputIndex].properties.connection !== '' && + this.props.batchInputs[this.props.selectedFlowBatchInputIndex].properties.path !== '')) && !this.props.fetchingInputSchema && this.props.getInputSchemaButtonEnabled; + const samplingInputLabel = this.renderSamplingInputLabel(this.props.input.mode); + return (
{display} -
- -
-
- this.props.onUpdateSamplingInputDuration(value)} - onGetErrorMessage={value => this.validateNumber(value)} - /> -
+ {samplingInputLabel}
); } @@ -386,7 +561,12 @@ export default class InputSettingsContent extends React.Component { let editor; if (this.props.fetchingInputSchema) { const timer = parseInt(this.props.samplingInputDuration) - this.props.timer; - const label = timer > -1 ? `Sampling Data... ${timer}` : 'Generating schema...'; + const label = + this.props.input.mode === Models.inputModeEnum.streaming + ? timer > -1 + ? `Sampling Data... ${timer}` + : 'Generating schema...' + : 'Reading blobs...'; editor = ; } else { @@ -439,9 +619,8 @@ export default class InputSettingsContent extends React.Component { View Example
-
- this.renderCosmosDbSettings(sinker), [Models.sinkerTypeEnum.eventHub]: sinker => this.renderEventHubSettings(sinker), [Models.sinkerTypeEnum.blob]: sinker => this.renderBlobSettings(sinker), + [Models.sinkerTypeEnum.sql]: sinker => this.renderSqlSettings(sinker), [Models.sinkerTypeEnum.metric]: sinker => this.renderMetricSettings(sinker), [Models.sinkerTypeEnum.local]: sinker => this.renderLocalSettings(sinker) }; @@ -123,14 +125,14 @@ export default class OutputSettingsContent extends React.Component { renderAddSinkerButton() { const menuItems = this.props.enableLocalOneBox ? Models.outputSinkerTypes - .filter(sinkerType => sinkerType.name == 'Local') + .filter(sinkerType => sinkerType.name === 'Local') .map(sinkerType => { return Object.assign({}, sinkerType, { onClick: () => this.props.onNewSinker(sinkerType.key) }); }) : Models.outputSinkerTypes - .filter(sinkerType => sinkerType.name != 'Local') + .filter(sinkerType => sinkerType.name !== 'Local') .map(sinkerType => { return Object.assign({}, sinkerType, { onClick: () => this.props.onNewSinker(sinkerType.key) @@ -233,6 +235,20 @@ export default class OutputSettingsContent extends React.Component { ); } + renderSqlSettings(sinker) { + return ( + + ); + } + renderMetricSettings(sinker) { return ( @@ -308,6 +324,12 @@ OutputSettingsContent.propTypes = { onUpdateBlobPrefix: PropTypes.func.isRequired, onUpdateBlobPartitionFormat: PropTypes.func.isRequired, + // Sql + onUpdateSqlConnection: PropTypes.func.isRequired, + onUpdateSqlTableName: PropTypes.func.isRequired, + onUpdateSqlWriteMode: PropTypes.func.isRequired, + onUpdateSqlUseBulkInsert: PropTypes.func.isRequired, + addOutputSinkButtonEnabled: PropTypes.bool.isRequired, deleteOutputSinkButtonEnabled: PropTypes.bool.isRequired }; diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/output/sqlSinkerSettings.jsx b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/output/sqlSinkerSettings.jsx new file mode 100644 index 000000000..c6c7b87cf --- /dev/null +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/output/sqlSinkerSettings.jsx @@ -0,0 +1,149 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +import React from 'react'; +import PropTypes from 'prop-types'; +import * as Helpers from '../../flowHelpers'; +import * as Models from '../../flowModels'; +import { TextField, Label, Dropdown, Toggle } from 'office-ui-fabric-react'; +import { Colors } from 'datax-common'; + +export default class SqlSinkerSettings extends React.Component { + constructor(props) { + super(props); + } + + render() { + return ( +
+
+ this.props.onUpdateSinkerName(value)} + onGetErrorMessage={value => this.validateProperty(value)} + /> +
+ +
+ +
+ +
+ this.props.onUpdateSqlConnection(value)} + autoAdjustHeight + resizable={false} + /> +
+ +
+ this.props.onUpdateSqlTableName(value)} + /> +
+ + {this.renderSqlWriteModeDropdown()} + {this.renderSqlBulkInsert()} +
+ ); + } + + renderSqlWriteModeDropdown() { + const isbulkInsert = this.props.sinker.properties.useBulkInsert; + const options = Models.sinkerSqlWriteModes.map(type => { + return { + key: type.key, + text: type.name, + disabled: type.disabled + }; + }); + + return ( +
+ + this.props.onUpdateSqlWriteMode(selection.key)} + disabled={isbulkInsert} + /> +
+ ); + } + + renderSqlBulkInsert() { + const isbulkInsert = this.props.sinker.properties.useBulkInsert; + return ( +
+
+ this.props.onUpdateSqlUseBulkInsert(value)} + /> +
+
+ ); + } + + validateProperty(value) { + if (value === '') return ''; + return !Helpers.isNumberAndStringOnly(value) ? 'Letters, numbers, and underscores only' : ''; + } +} + +// Props +SqlSinkerSettings.propTypes = { + sinker: PropTypes.object.isRequired, + sinkerDisplayName: PropTypes.string.isRequired, + + onUpdateSinkerName: PropTypes.func.isRequired, + onUpdateSqlConnection: PropTypes.func.isRequired, + onUpdateSqlTableName: PropTypes.func.isRequired, + onUpdateSqlWriteMode: PropTypes.func.isRequired, + onUpdateSqlUseBulkInsert: PropTypes.func.isRequired +}; + +// Styles +const rootStyle = { + paddingLeft: 30, + paddingRight: 30, + paddingBottom: 30 +}; + +const sinkTypeSection = { + paddingBottom: 40 +}; + +const sectionStyle = { + paddingBottom: 15 +}; + +const toggleSectionStyle = { + paddingBottom: 29, + paddingRight: 15, + width: 200, + minWidth: 200 +}; diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/scale/scaleSettingsContent.jsx b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/scale/scaleSettingsContent.jsx index 0bc43c1b0..4b590744c 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/scale/scaleSettingsContent.jsx +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/scale/scaleSettingsContent.jsx @@ -4,7 +4,7 @@ // ********************************************************************* import React from 'react'; import PropTypes from 'prop-types'; -import { Slider } from 'office-ui-fabric-react'; +import { Slider, Toggle } from 'office-ui-fabric-react'; import { Colors, ScrollableContentPane, StatementBox } from 'datax-common'; export default class ScaleSettingsContent extends React.Component { @@ -22,6 +22,15 @@ export default class ScaleSettingsContent extends React.Component { } renderContent() { + return ( +
+ {this.props.isDatabricksSparkType && this.renderContentForDatabricks()} + {!this.props.isDatabricksSparkType && this.renderContentContentForHDInsight()} +
+ ); + } + + renderContentContentForHDInsight() { return (
@@ -54,6 +63,51 @@ export default class ScaleSettingsContent extends React.Component {
); } + + renderContentForDatabricks() { + return ( +
+
+ this.props.onUpdateDatabricksAutoScale(value)} + /> +
+ +
+ this.props.onUpdateDatabricksMinWorkers(value.toString())} + /> +
+ + {this.props.scale.jobDatabricksAutoScale && ( +
+ this.props.onUpdateDatabricksMaxWorkers(value.toString())} + /> +
+ )} +
+ ); + } } // Props @@ -62,6 +116,9 @@ ScaleSettingsContent.propTypes = { onUpdateNumExecutors: PropTypes.func.isRequired, onUpdateExecutorMemory: PropTypes.func.isRequired, + onUpdateDatabricksAutoScale: PropTypes.func.isRequired, + onUpdateDatabricksMinWorkers: PropTypes.func.isRequired, + onUpdateDatabricksMaxWorkers: PropTypes.func.isRequired, scaleNumExecutorsSliderEnabled: PropTypes.bool.isRequired, scaleExecutorMemorySliderEnabled: PropTypes.bool.isRequired @@ -83,3 +140,8 @@ const contentStyle = { const sectionStyle = { paddingBottom: 15 }; + +const toggleSectionStyle = { + paddingTop: 10, + paddingBottom: 15 +}; diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/schedule/oneTimeScheduleSettings.jsx b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/schedule/oneTimeScheduleSettings.jsx new file mode 100644 index 000000000..f445f307f --- /dev/null +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/schedule/oneTimeScheduleSettings.jsx @@ -0,0 +1,205 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +import React from 'react'; +import PropTypes from 'prop-types'; +import * as Helpers from '../../flowHelpers'; +import * as Models from '../../flowModels'; +import { TextField, Label, Dropdown } from 'office-ui-fabric-react'; +import DateTimePicker from 'react-datetime-picker'; + +export default class OneTimeScheduleSettings extends React.Component { + constructor(props) { + super(props); + } + + render() { + return ( +
+ {this.renderAlias()} + {this.renderTypeDisplayName()} + {this.renderInterval()} + {this.renderWindow()} + {this.renderTimeRange()} +
+ ); + } + + renderAlias() { + return ( +
+ this.props.onUpdateBatchName(value)} + onGetErrorMessage={value => this.validateProperty(value)} + /> +
+ ); + } + + renderTypeDisplayName() { + return ( +
+ +
+ ); + } + + renderTimeRange() { + return ( +
+ {this.renderStartTime()} + {this.renderEndTime()} +
+ ); + } + + renderStartTime() { + let maxDate = this.props.batch.properties.endTime ? new Date(this.props.batch.properties.endTime) : ''; + + return ( +
+ + this.props.onUpdateBatchStartTime(value)} + /> +
+ ); + } + + renderEndTime() { + let minDate = this.props.batch.properties.startTime ? new Date(this.props.batch.properties.startTime) : ''; + + return ( +
+ + this.props.onUpdateBatchEndTime(value)} + /> +
+ ); + } + + renderInterval() { + return ( +
+ {this.renderBatchSettingValue('Interval', this.props.batch.properties.interval, this.props.onUpdateBatchIntervalValue)} + {this.renderBatchSettingUnitType(this.props.batch.properties.intervalType, this.props.onUpdateBatchIntervalType)} +
+ ); + } + + renderWindow() { + return ( +
+ {this.renderBatchSettingValue('Window', this.props.batch.properties.window, this.props.onUpdateBatchWindowValue)} + {this.renderBatchSettingUnitType(this.props.batch.properties.windowType, this.props.onUpdateBatchWindowType)} +
+ ); + } + + renderBatchSettingValue(type, value, onUpdateValue) { + return ( +
+ onUpdateValue(value)} + /> +
+ ); + } + + renderBatchSettingUnitType(value, onUpdateType) { + const options = Models.batchIntervalTypes.map(type => { + return { + key: type.key, + text: type.name, + disabled: type.disabled + }; + }); + + return ( +
+ + onUpdateType(selection.key)} + /> +
+ ); + } + + validateProperty(value) { + if (value === '') return ''; + return !Helpers.isNumberAndStringOnly(value) ? 'Letters, numbers, and underscores only' : ''; + } +} + +// Props +OneTimeScheduleSettings.propTypes = { + batch: PropTypes.object.isRequired, + // functions + onUpdateBatchName: PropTypes.func.isRequired, + onUpdateBatchStartTime: PropTypes.func.isRequired, + onUpdateBatchEndTime: PropTypes.func.isRequired, + onUpdateBatchIntervalValue: PropTypes.func.isRequired, + onUpdateBatchIntervalType: PropTypes.func.isRequired, + onUpdateBatchWindowValue: PropTypes.func.isRequired, + onUpdateBatchWindowType: PropTypes.func.isRequired +}; + +// Styles +const rootStyle = { + paddingLeft: 30, + paddingRight: 30, + paddingBottom: 30 +}; + +const sectionStyle = { + paddingBottom: 15 +}; + +const batchTypeSection = { + paddingBottom: 40 +}; + +const sectionContainerStyle = { + display: 'flex', + flexDirection: 'row', + width: 400 +}; + +const sectionValueStyle = { + flex: 1, + marginRight: 10 +}; + +const sectionDropdownStyle = { + flex: 1, + paddingBottom: 15 +}; diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/schedule/recurringScheduleSettings.jsx b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/schedule/recurringScheduleSettings.jsx new file mode 100644 index 000000000..905b9a4b7 --- /dev/null +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/schedule/recurringScheduleSettings.jsx @@ -0,0 +1,219 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +import React from 'react'; +import PropTypes from 'prop-types'; +import * as Helpers from '../../flowHelpers'; +import * as Models from '../../flowModels'; +import { TextField, Label, Dropdown } from 'office-ui-fabric-react'; +import DateTimePicker from 'react-datetime-picker'; + +export default class RecurringScheduleSettings extends React.Component { + constructor(props) { + super(props); + } + + render() { + return ( +
+ {this.renderAlias()} + {this.renderTypeDisplayName()} + {this.renderInterval()} + {this.renderDelay()} + {this.renderWindow()} + {this.renderTimeRange()} +
+ ); + } + + renderAlias() { + return ( +
+ this.props.onUpdateBatchName(value)} + onGetErrorMessage={value => this.validateProperty(value)} + /> +
+ ); + } + + renderTypeDisplayName() { + return ( +
+ +
+ ); + } + + renderTimeRange() { + return ( +
+ {this.renderStartTime()} + {this.renderEndTime()} +
+ ); + } + + renderStartTime() { + let maxDate = this.props.batch.properties.endTime ? new Date(this.props.batch.properties.endTime) : ''; + + return ( +
+ + this.props.onUpdateBatchStartTime(value)} + /> +
+ ); + } + + renderEndTime() { + let minDate = this.props.batch.properties.startTime ? new Date(this.props.batch.properties.startTime) : ''; + + return ( +
+ + this.props.onUpdateBatchEndTime(value)} + /> +
+ ); + } + + renderInterval() { + return ( +
+ {this.renderBatchSettingValue('Recurrence', this.props.batch.properties.interval, this.props.onUpdateBatchIntervalValue)} + {this.renderBatchSettingUnitType(this.props.batch.properties.intervalType, this.props.onUpdateBatchIntervalType)} +
+ ); + } + + renderDelay() { + return ( +
+ {this.renderBatchSettingValue('Delay', this.props.batch.properties.delay, this.props.onUpdateBatchDelayValue)} + {this.renderBatchSettingUnitType(this.props.batch.properties.delayType, this.props.onUpdateBatchDelayType)} +
+ ); + } + + renderWindow() { + return ( +
+ {this.renderBatchSettingValue('Window', this.props.batch.properties.window, this.props.onUpdateBatchWindowValue)} + {this.renderBatchSettingUnitType(this.props.batch.properties.windowType, this.props.onUpdateBatchWindowType)} +
+ ); + } + + renderBatchSettingValue(type, value, onUpdateValue) { + return ( +
+ onUpdateValue(value)} + /> +
+ ); + } + + renderBatchSettingUnitType(value, onUpdateType) { + const options = Models.batchIntervalTypes.map(type => { + return { + key: type.key, + text: type.name, + disabled: type.disabled + }; + }); + + return ( +
+ + onUpdateType(selection.key)} + /> +
+ ); + } + + validateProperty(value) { + if (value === '') return ''; + return !Helpers.isNumberAndStringOnly(value) ? 'Letters, numbers, and underscores only' : ''; + } +} + +// Props +RecurringScheduleSettings.propTypes = { + batch: PropTypes.object.isRequired, + batchTypeDisplayName: PropTypes.string.isRequired, + + // functions + onUpdateBatchName: PropTypes.func.isRequired, + onUpdateBatchStartTime: PropTypes.func.isRequired, + onUpdateBatchEndTime: PropTypes.func.isRequired, + onUpdateBatchIntervalValue: PropTypes.func.isRequired, + onUpdateBatchIntervalType: PropTypes.func.isRequired, + onUpdateBatchDelayValue: PropTypes.func.isRequired, + onUpdateBatchDelayType: PropTypes.func.isRequired, + onUpdateBatchWindowValue: PropTypes.func.isRequired, + onUpdateBatchWindowType: PropTypes.func.isRequired +}; + +// Styles +const rootStyle = { + paddingLeft: 30, + paddingRight: 30, + paddingBottom: 30 +}; + +const sectionStyle = { + paddingBottom: 15 +}; + +const batchTypeSection = { + paddingBottom: 40 +}; + +const sectionContainerStyle = { + display: 'flex', + flexDirection: 'row', + width: 400 +}; + +const sectionValueStyle = { + flex: 1, + marginRight: 10 +}; + +const sectionDropdownStyle = { + flex: 1, + paddingBottom: 15 +}; diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/schedule/scheduleSettingsContent.jsx b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/schedule/scheduleSettingsContent.jsx new file mode 100644 index 000000000..8672e7263 --- /dev/null +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/schedule/scheduleSettingsContent.jsx @@ -0,0 +1,326 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +import React from 'react'; +import PropTypes from 'prop-types'; +import * as Models from '../../flowModels'; +import { DetailsList, DetailsListLayoutMode, Selection, SelectionMode, CheckboxVisibility, DefaultButton } from 'office-ui-fabric-react'; +import { Colors, IconButtonStyles, PanelHeader, PanelHeaderButtons, ScrollableContentPane, StatementBox } from 'datax-common'; +import RecurringScheduleSettings from './recurringScheduleSettings'; +import OneTimeScheduleSettings from './oneTimeScheduleSettings'; +import * as Styles from '../../../../common/styles'; + +const batchColumns = [ + { + key: 'columnBatchName', + name: 'Alias', + fieldName: 'id', + isResizable: true + }, + { + key: 'columnBatchType', + name: 'Type', + fieldName: 'typeDisplay', + isResizable: true, + minWidth: 100 + }, + { + key: 'columnBatchStatus', + name: 'Status', + fieldName: 'status', + isResizable: true, + minWidth: 100 + } +]; + +export default class ScheduleSettingsContent extends React.Component { + constructor(props) { + super(props); + + this.batchSelection = new Selection({ + selectionMode: SelectionMode.single + }); + + this.batchTypeToRenderFuncMap = { + [Models.batchTypeEnum.recurring]: batch => this.renderRecurringScheduleSettings(batch), + [Models.batchTypeEnum.oneTime]: batch => this.renderOneTimeScheduleSettings(batch) + }; + + this.batchTypeToDisplayMap = {}; + Models.batchTypes.forEach(batchType => { + this.batchTypeToDisplayMap[batchType.key] = batchType.name; + }); + } + + render() { + return ( +
+ + {this.renderContent()} +
+ ); + } + + renderContent() { + return ( +
+ {this.renderLeftPane()} + {this.renderRightPane()} +
+ ); + } + + renderLeftPane() { + this.showSelectionInDetailsList(); + + const batchList = this.props.batchList; + batchList.forEach(item => { + item.typeDisplay = this.batchTypeToDisplayMap[item.type]; + item.status = !item.disabled ? 'Active' : 'Disabled'; + }); + + return ( +
+ Schedule Batch Jobs + {this.renderButtons()} + +
+
+ this.onSelectBatchItem(item, index)} + setKey="batchList" + layoutMode={DetailsListLayoutMode.justified} + selectionMode={SelectionMode.single} + selectionPreservedOnEmptyClick={true} + isHeaderVisible={true} + checkboxVisibility={CheckboxVisibility.hidden} + selection={this.batchSelection} + /> +
+
+
+ ); + } + + renderRightPane() { + return ( +
+ Settings + {this.renderBatchSettings()} +
+ ); + } + + renderButtons() { + return [this.renderAddBatchButton(), this.renderDeleteBatchButton()]; + } + + renderAddBatchButton() { + let recurringExists = this.props.batchList.filter(b => b.type === Models.batchTypeEnum.recurring); + + const menuItems = + !recurringExists || recurringExists.length < 1 + ? Models.batchTypes.map(batchType => { + return Object.assign({}, batchType, { + onClick: () => this.props.onNewBatch(batchType.key) + }); + }) + : Models.batchTypes + .filter(batchType => batchType.key !== Models.batchTypeEnum.recurring) + .map(batchType => { + return Object.assign({}, batchType, { + onClick: () => this.props.onNewBatch(batchType.key) + }); + }); + + return ( + + + Add + + ); + } + + renderDeleteBatchButton() { + const batch = this.getBatch(); + const enableButton = batch !== undefined && this.props.deleteBatchButtonEnabled; + + return ( + this.props.onDeleteBatch(this.props.selectedBatchIndex)} + > + + Delete + + ); + } + + renderBatchSettings() { + const batch = this.getBatch(); + if (!batch) { + return null; + } + + if (batch.type in this.batchTypeToRenderFuncMap) { + return this.batchTypeToRenderFuncMap[batch.type](batch); + } else { + alert('Not supported batch type'); + return null; + } + } + + renderRecurringScheduleSettings(batch) { + return ( + + ); + } + + renderOneTimeScheduleSettings(batch) { + return ( + + ); + } + + onSelectBatchItem(item, index) { + if (index !== this.props.selectedBatchIndex) { + this.props.onUpdateSelectedBatchIndex(index); + } + } + + getBatch() { + const batchList = this.props.batchList; + if (this.props.selectedBatchIndex !== undefined && this.props.selectedBatchIndex < batchList.length) { + return batchList[this.props.selectedBatchIndex]; + } else { + return undefined; + } + } + + showSelectionInDetailsList() { + const batchList = this.props.batchList; + if (batchList.length > 0) { + this.batchSelection.setChangeEvents(false, true); + this.batchSelection.setItems(batchList); + this.batchSelection.setIndexSelected(this.props.selectedBatchIndex, true, false); + this.batchSelection.setChangeEvents(true, true); + } + } +} + +// Props +ScheduleSettingsContent.propTypes = { + batchList: PropTypes.array.isRequired, + selectedBatchIndex: PropTypes.number, + + onNewBatch: PropTypes.func.isRequired, + onDeleteBatch: PropTypes.func.isRequired, + onUpdateSelectedBatchIndex: PropTypes.func.isRequired, + + addBatchButtonEnabled: PropTypes.bool.isRequired, + deleteBatchButtonEnabled: PropTypes.bool.isRequired, + + // Schedule Settings + onUpdateBatchName: PropTypes.func.isRequired, + onUpdateBatchStartTime: PropTypes.func.isRequired, + onUpdateBatchEndTime: PropTypes.func.isRequired, + onUpdateBatchIntervalValue: PropTypes.func.isRequired, + onUpdateBatchIntervalType: PropTypes.func.isRequired, + onUpdateBatchDelayValue: PropTypes.func.isRequired, + onUpdateBatchDelayType: PropTypes.func.isRequired, + onUpdateBatchWindowValue: PropTypes.func.isRequired, + onUpdateBatchWindowType: PropTypes.func.isRequired +}; + +// Styles +const rootStyle = { + display: 'flex', + flexDirection: 'column', + overflowY: 'hidden', + flex: 1 +}; + +const contentStyle = { + display: 'flex', + flexDirection: 'row', + overflowY: 'hidden', + flex: 1 +}; + +const panelHeaderButtonStyle = { + backgroundColor: Colors.neutralLighter +}; + +const leftPaneStyle = { + flex: 1, + display: 'flex', + flexDirection: 'column', + borderRight: `1px solid ${Colors.neutralTertiaryAlt}` +}; + +const leftPaneContentStyle = { + paddingLeft: 30, + paddingRight: 30, + paddingTop: 30, + paddingBottom: 30, + flex: 1, + display: 'flex', + flexDirection: 'column', + overflowY: 'hidden' +}; + +const rightPaneStyle = { + flex: 3, + display: 'flex', + flexDirection: 'column' +}; + +const listContainerStyle = { + backgroundColor: Colors.white, + border: `1px solid ${Colors.neutralTertiaryAlt}`, + flex: 1, + overflowY: 'auto' +}; diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowActions.js b/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowActions.js index cdc76645a..4bf991195 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowActions.js +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowActions.js @@ -3,14 +3,13 @@ // Licensed under the MIT License // ********************************************************************* import Q from 'q'; -import { UserSelectors, getApiErrorMessage } from 'datax-common'; - import * as Api from './api'; -import * as Helpers from './flowHelpers'; +import { isDatabricksSparkType } from '../../common/api'; import * as Selectors from './flowSelectors'; -import * as KernelActions from './kernelActions'; -import * as KernelSelectors from './kernelSelectors'; - +import { UserSelectors, getApiErrorMessage } from 'datax-common'; +import { QueryActions, QueryModels } from 'datax-query'; +import * as Helpers from './flowHelpers'; +import * as Models from './flowModels'; /** * * REDUX Action Types @@ -24,9 +23,8 @@ export const FLOW_NEW = 'FLOW_NEW'; // Info export const FLOW_UPDATE_DISPLAY_NAME = 'FLOW_UPDATE_DISPLAY_NAME'; export const FLOW_UPDATE_OWNER = 'FLOW_UPDATE_OWNER'; - -// Query -export const FLOW_UPDATE_QUERY = 'FLOW_UPDATE_QUERY'; +export const FLOW_UPDATE_DATABRICKSTOKEN = 'FLOW_UPDATE_DATABRICKSTOKEN'; +export const FLOW_UPDATE_ISDATABRICKSSPARKTYPE = 'FLOW_UPDATE_ISDATABRICKSSPARKTYPE'; // Scale export const FLOW_UPDATE_SCALE = 'FLOW_UPDATE_SCALE'; @@ -38,6 +36,12 @@ export const FLOW_DELETE_SINKER = 'FLOW_DELETE_SINKER'; export const FLOW_UPDATE_SINKER = 'FLOW_UPDATE_SINKER'; export const FLOW_UPDATE_SELECTED_SINKER_INDEX = 'FLOW_UPDATE_SELECTED_SINKER_INDEX'; +// Schedule +export const FLOW_NEW_BATCH = 'FLOW_NEW_BATCH'; +export const FLOW_DELETE_BATCH = 'FLOW_DELETE_BATCH'; +export const FLOW_UPDATE_BATCHLIST = 'FLOW_UPDATE_BATCHLIST'; +export const FLOW_UPDATE_SELECTED_BATCH_INDEX = 'FLOW_UPDATE_SELECTED_BATCH_INDEX'; + // Rules export const FLOW_UPDATE_RULES = 'FLOW_UPDATE_RULES'; export const FLOW_NEW_RULE = 'FLOW_NEW_RULE'; @@ -61,9 +65,9 @@ export const FLOW_UPDATE_SELECTED_FUNCTION_INDEX = 'FLOW_UPDATE_SELECTED_FUNCTIO // Input export const FLOW_UPDATE_INPUT = 'FLOW_UPDATE_INPUT'; +export const FLOW_UPDATE_BATCH_INPUT = 'FLOW_UPDATE_BATCH_INPUT'; export const FLOW_FETCHING_INPUT_SCHEMA = 'FLOW_FETCHING_INPUT_SCHEMA'; export const FLOW_UPDATE_SAMPLING_INPUT_DURATION = 'FLOW_UPDATE_SAMPLING_INPUT_DURATION'; -export const FLOW_UPDATE_RESAMPLING_INPUT_DURATION = 'FLOW_UPDATE_RESAMPLING_INPUT_DURATION'; // Message export const FLOW_UPDATE_ERROR_MESSAGE = 'FLOW_UPDATE_ERROR_MESSAGE'; @@ -78,6 +82,15 @@ export const FLOW_UPDATE_ONEBOX_MODE = 'FLOW_UPDATE_ONEBOX_MODE'; * */ +function getSparkEnvAndUpdateIsDatabricksSparkType(dispatch){ + isDatabricksSparkType().then(response => { + return dispatch({ + type: FLOW_UPDATE_ISDATABRICKSSPARKTYPE, + payload: response + }); + }); +} + // Init Actions export const initFlow = context => (dispatch, getState) => { if (context && context.id) { @@ -89,6 +102,10 @@ export const initFlow = context => (dispatch, getState) => { payload: flow }); }) + .then(flow => { + dispatch(QueryActions.initQuery(flow.payload.query)); + getSparkEnvAndUpdateIsDatabricksSparkType(dispatch); + }) .catch(error => { const message = getApiErrorMessage(error); updateErrorMessage(dispatch, message); @@ -96,6 +113,8 @@ export const initFlow = context => (dispatch, getState) => { }); } else { const owner = UserSelectors.getUserAlias(getState()); + dispatch(QueryActions.initQuery(QueryModels.defaultQuery)); + getSparkEnvAndUpdateIsDatabricksSparkType(dispatch); return dispatch({ type: FLOW_NEW, payload: owner @@ -118,12 +137,25 @@ export const updateOwner = () => (dispatch, getState) => { }); }; +export const updateDatabricksToken = databricksToken => dispatch => { + return dispatch({ + type: FLOW_UPDATE_DATABRICKSTOKEN, + payload: databricksToken + }); +}; + // Input Actions export const updateInputMode = mode => (dispatch, getState) => { + let type = mode === Models.inputModeEnum.streaming ? Models.inputTypeEnum.events : Models.inputTypeEnum.blob; + let snippet = Models.getDefaultNormalizationSnippet(mode); updateInput( dispatch, Object.assign({}, Selectors.getFlowInput(getState()), { - mode: mode + mode: mode, + type: type, + properties: Object.assign({}, Selectors.getFlowInputProperties(getState()), { + normalizationSnippet: snippet + }) }) ); }; @@ -143,6 +175,12 @@ export const updateInputHubName = name => (dispatch, getState) => { }); }; +export const updateInputPath = path => (dispatch, getState) => { + updateInputProperties(dispatch, getState, { + inputPath: path + }); +}; + export const updateInputHubConnection = connection => (dispatch, getState) => { updateInputProperties(dispatch, getState, { inputEventhubConnection: connection @@ -161,6 +199,48 @@ export const updateInputResourceGroup = name => (dispatch, getState) => { }); }; +export const updateBatchInputConnection = connection => (dispatch, getState) => { + updateBatchInputProperties(dispatch, getState, { + connection: connection + }); +}; + +export const updateBlobInputPath = path => (dispatch, getState) => { + updateBatchInputProperties(dispatch, getState, { + path: path + }); +}; + +export const updateBatchInputFormatType = formatType => (dispatch, getState) => { + updateBatchInputProperties(dispatch, getState, { + formatType: formatType + }); +}; + +export const updateBachInputCompressionType = compressionType => (dispatch, getState) => { + updateBatchInputProperties(dispatch, getState, { + compressionType: compressionType + }); +}; + +function updateBatchInputProperties(dispatch, getState, propertyMember) { + updateBatchInput( + dispatch, + Selectors.getSelectedBatchInputIndex(getState()), + Object.assign({}, Selectors.getSelectedBatchInput(getState()), { + properties: Object.assign({}, Selectors.getSelectedBatchInputProperties(getState()), propertyMember) + }) + ); +} + +function updateBatchInput(dispatch, index, batchInput) { + return dispatch({ + type: FLOW_UPDATE_BATCH_INPUT, + payload: batchInput, + index: index + }); +} + export const updateInputWindowDuration = duration => (dispatch, getState) => { updateInputProperties(dispatch, getState, { windowDuration: duration @@ -210,13 +290,6 @@ export const updateSamplingInputDuration = duration => dispatch => { }); }; -export const updateResamplingInputDuration = duration => dispatch => { - return dispatch({ - type: FLOW_UPDATE_RESAMPLING_INPUT_DURATION, - duration: duration - }); -}; - export const updateInputSchema = schema => (dispatch, getState) => { updateInputProperties(dispatch, getState, { inputSchemaFile: schema @@ -455,68 +528,6 @@ function updateFunction(dispatch, index, functionItem) { }); } -// Query Actions -export const updateQuery = query => dispatch => { - return dispatch({ - type: FLOW_UPDATE_QUERY, - payload: query - }); -}; - -export const getTableSchemas = flow => { - return Api.getTableSchemas(flow).then(tables => { - let tableToSchemaMap = {}; - tables.forEach(table => { - tableToSchemaMap[table.name] = table; - }); - - return tableToSchemaMap; - }); -}; - -export const getCodeGenQuery = flow => { - return Api.getCodeGenQuery(flow).then(query => { - return query; - }); -}; - -export const executeQuery = (flow, selectedQuery, kernelId) => dispatch => { - updateErrorMessage(dispatch, undefined); - return Api.executeQuery(flow, selectedQuery, kernelId) - .then(result => { - return result; - }) - .catch(error => { - const message = getApiErrorMessage(error); - updateErrorMessage(dispatch, message); - return Q.reject({ error: true, message: message }); - }); -}; - -export const resampleInput = (flow, kernelId, version) => (dispatch, getState) => { - updateErrorMessage(dispatch, undefined); - KernelActions.fetchingKernel(dispatch, true); - return Api.resampleInput(flow, kernelId) - .then(response => { - const kernelId = response.result; - const warning = response.message; - - const curVersion = KernelSelectors.getKernelVersion(getState()); - - if (version >= curVersion) { - return KernelActions.updateKernel(dispatch, kernelId, version, warning); - } else { - return Api.deleteDiagnosticKernel(kernelId); - } - }) - .catch(error => { - const message = getApiErrorMessage(error); - updateErrorMessage(dispatch, message); - KernelActions.fetchingKernel(dispatch, false); - return Q.reject({ error: true, message: message }); - }); -}; - // Output Actions export const updateOutputs = outputs => dispatch => { return dispatch({ @@ -638,6 +649,31 @@ function updateSinker(dispatch, index, sinker) { }); } +// Sql Sinker +export const updateSqlConnection = connection => (dispatch, getState) => { + updateSinkerProperties(dispatch, getState, { + connectionString: connection + }); +}; + +export const updateSqlTableName = name => (dispatch, getState) => { + updateSinkerProperties(dispatch, getState, { + tableName: name + }); +}; + +export const updateSqlWriteMode = mode => (dispatch, getState) => { + updateSinkerProperties(dispatch, getState, { + writeMode: mode + }); +}; + +export const updateSqlUseBulkInsert = bulkInsert => (dispatch, getState) => { + updateSinkerProperties(dispatch, getState, { + useBulkInsert: bulkInsert + }); +}; + // Rule Actions export const updateRules = rules => dispatch => { return dispatch({ @@ -782,6 +818,33 @@ export const updateExecutorMemory = executorMemory => (dispatch, getState) => { ); }; +export const updateDatabricksAutoScale = databricksAutoScale => (dispatch, getState) => { + updateScale( + dispatch, + Object.assign({}, Selectors.getFlowScale(getState()), { + jobDatabricksAutoScale: databricksAutoScale + }) + ); +}; + +export const updateDatabricksMinWorkers = databricksMinWorkers => (dispatch, getState) => { + updateScale( + dispatch, + Object.assign({}, Selectors.getFlowScale(getState()), { + jobDatabricksMinWorkers: databricksMinWorkers + }) + ); +}; + +export const updateDatabricksMaxWorkers = databricksMaxWorkers => (dispatch, getState) => { + updateScale( + dispatch, + Object.assign({}, Selectors.getFlowScale(getState()), { + jobDatabricksMaxWorkers: databricksMaxWorkers + }) + ); +}; + function updateScale(dispatch, scale) { return dispatch({ type: FLOW_UPDATE_SCALE, @@ -811,8 +874,22 @@ const rejectWithMessage = (error, msg) => }); // Save and Delete Actions -export const saveFlow = flow => { - return Api.saveFlow(Helpers.convertFlowToConfig(flow)).then(result => { +export const saveFlow = (flow, query) => { + return Api.saveFlow(Helpers.convertFlowToConfig(flow, query)) + .then(result => { + return result.name; + }) + .catch(error => { + const message = getApiErrorMessage(error); + return Q.reject({ + error: true, + message: `There was an issue saving the Flow. Please fix following error then save again: ${message}` + }); + }); +}; + +export const deployFlow = (flow, query) => { + return Api.saveFlow(Helpers.convertFlowToConfig(flow, query)).then(result => { const name = result.name; // generate job configurations for product @@ -844,3 +921,101 @@ export const updateOneBoxMode = enableLocalOneBox => dispatch => { payload: enableLocalOneBox }); }; + +// Batch +export const newBatch = type => dispatch => { + return dispatch({ + type: FLOW_NEW_BATCH, + payload: type + }); +}; + +export const deleteBatch = index => dispatch => { + return dispatch({ + type: FLOW_DELETE_BATCH, + index: index + }); +}; + +export const updateSelectedBatchIndex = index => dispatch => { + return dispatch({ + type: FLOW_UPDATE_SELECTED_BATCH_INDEX, + payload: index + }); +}; + +export const updateBatchName = name => (dispatch, getState) => { + updateBatchList( + dispatch, + Selectors.getSelectedBatchIndex(getState()), + Object.assign({}, Selectors.getSelectedBatch(getState()), { + id: name + }) + ); +}; + +export const updateBatchStartTime = startTime => (dispatch, getState) => { + updateBatchProperties(dispatch, getState, { + startTime: startTime + }); +}; + +export const updateBatchEndTime = endTime => (dispatch, getState) => { + updateBatchProperties(dispatch, getState, { + endTime: endTime + }); +}; + +export const updateBatchIntervalValue = interval => (dispatch, getState) => { + updateBatchProperties(dispatch, getState, { + interval: interval + }); +}; + +export const updateBatchIntervalType = type => (dispatch, getState) => { + updateBatchProperties(dispatch, getState, { + intervalType: type + }); +}; + +export const updateBatchDelayValue = delay => (dispatch, getState) => { + updateBatchProperties(dispatch, getState, { + delay: delay + }); +}; + +export const updateBatchDelayType = type => (dispatch, getState) => { + updateBatchProperties(dispatch, getState, { + delayType: type + }); +}; + +export const updateBatchWindowValue = window => (dispatch, getState) => { + updateBatchProperties(dispatch, getState, { + window: window + }); +}; + +export const updateBatchWindowType = type => (dispatch, getState) => { + updateBatchProperties(dispatch, getState, { + windowType: type + }); +}; + +function updateBatchProperties(dispatch, getState, propertyMember) { + updateBatchList( + dispatch, + Selectors.getSelectedBatchIndex(getState()), + Object.assign({}, Selectors.getSelectedBatch(getState()), { + properties: Object.assign({}, Selectors.getSelectedBatchProperties(getState()), propertyMember) + }) + ); +} + +function updateBatchList(dispatch, index, batch) { + return dispatch({ + type: FLOW_UPDATE_BATCHLIST, + payload: batch, + index: index + }); +} diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowHelpers.js b/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowHelpers.js index deb741f8c..1894cd926 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowHelpers.js +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowHelpers.js @@ -3,13 +3,7 @@ // Licensed under the MIT License // ********************************************************************* import * as Models from './flowModels'; - -export function isValidNumberAboveZero(value) { - const number = Number(value); - const isNumber = !isNaN(number); - const isValid = isNumber && number > 0 && value[0] !== '0'; - return isValid; -} +import * as Api from '../../common/api'; export function isValidNumberAboveOrEqualZero(value) { const number = Number(value); @@ -441,12 +435,14 @@ export function isMetricSinker(sinker) { return sinker.type === Models.sinkerTypeEnum.metric; } -export function convertFlowToConfig(flow) { +export function convertFlowToConfig(flow, query) { let referenceData = [...flow.referenceData]; let functions = [...flow.functions]; let sinkers = [...flow.outputs]; let outputTemplates = [...flow.outputTemplates]; let rules = [...flow.rules]; + let batchList = flow.batchList ? [...flow.batchList] : []; + let batchInputs = flow.batchInputs ? [...flow.batchInputs] : [Models.getDefaultBatchInputSettings()]; // sort by name referenceData.sort((a, b) => a.id.localeCompare(b.id)); @@ -454,23 +450,27 @@ export function convertFlowToConfig(flow) { sinkers.sort((a, b) => a.id.localeCompare(b.id)); outputTemplates.sort((a, b) => a.id.localeCompare(b.id)); rules.sort((a, b) => a.id.localeCompare(b.id)); + batchList.sort((b, a) => a.type.localeCompare(b.type)); // return product config return { name: flow.name, + flowId: flow.flowId, displayName: flow.displayName.trim(), owner: flow.owner, - input: Object.assign({}, flow.input, { referenceData: flow.referenceData }), + databricksToken: flow.databricksToken, + input: Object.assign({}, flow.input, { referenceData: flow.referenceData, batch: batchInputs }), process: { timestampColumn: flow.input.properties.timestampColumn, watermark: `${flow.input.properties.watermarkValue} ${flow.input.properties.watermarkUnit}`, functions: functions, - queries: [flow.query], + queries: [query], jobconfig: flow.scale }, outputs: sinkers, outputTemplates: outputTemplates, - rules: convertFlowToConfigRules(rules) + rules: convertFlowToConfigRules(rules), + batchList: batchList }; } @@ -489,9 +489,13 @@ export function convertConfigToFlow(config) { // return flow understood by our website let flow = { name: config.name, + flowId: config.flowId, displayName: config.displayName, owner: config.owner, + databricksToken: config.databricksToken, input: input, + batchInputs: input.batch ? input.batch : [Models.getDefaultBatchInputSettings()], + batchList: config.batchList ? config.batchList : [], referenceData: input.referenceData ? input.referenceData : [], functions: config.process.functions ? config.process.functions : [], query: config.process.queries[0], @@ -503,3 +507,27 @@ export function convertConfigToFlow(config) { return flow; } +/** This is the contract between the package that is dependent on datax-query package. For example, datax-pipeline package needs to pass in queryMetadata object which contains all the required parameters as needed by various apis in the datax-query package*/ +export function convertFlowToQueryMetadata(flow, query) { + // return query metadata + let QueryMetadata = { + name: flow.name, + databricksToken: flow.databricksToken, + displayName: flow.displayName, + userName: flow.owner, + refData: flow.referenceData, + inputSchema: flow.input.properties.inputSchemaFile, + normalizationSnippet: flow.input.properties.normalizationSnippet, + outputTemplates: flow.outputTemplates, + functions: flow.functions, + rules: convertFlowToConfigRules(flow.rules), + eventhubConnection: flow.input.properties.inputEventhubConnection, + inputResourceGroup: flow.input.properties.inputResourceGroup, + eventhubNames: flow.input.properties.inputEventhubName, + inputType: flow.input.type, + seconds: flow.resamplingInputDuration, + query: query, + inputSubscriptionId: flow.input.properties.inputSubscriptionId + }; + return QueryMetadata; +} diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowModels.js b/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowModels.js index 5810ae910..076add958 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowModels.js +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowModels.js @@ -3,7 +3,14 @@ // Licensed under the MIT License // ********************************************************************* export const inputModeEnum = { - streaming: 'streaming' + streaming: 'streaming', + batching: 'batching' +}; + +export const batchIntervalTypeEnum = { + day: 'day', + hour: 'hour', + min: 'min' }; export const inputTypeEnum = { @@ -11,9 +18,20 @@ export const inputTypeEnum = { iothub: 'iothub', kafka: 'kafka', kafkaeventhub: 'kafkaeventhub', + blob: 'blob', local: 'local' }; +export const inputCompressionTypeEnum = { + none: 'none', + gzip: 'gzip' +}; + +export const inputFormatTypeEnum = { + json: 'json', + parquet: 'parquet' +}; + export const watermarkUnitEnum = { second: 'second', minute: 'minute', @@ -45,7 +63,13 @@ export const sinkerTypeEnum = { eventHub: 'eventHub', blob: 'blob', metric: 'metric', - local: 'local' + local: 'local', + sql: 'sqlServer' +}; + +export const batchTypeEnum = { + recurring: 'recurring', + oneTime: 'oneTime' }; export const sinkerCompressionTypeEnum = { @@ -57,6 +81,36 @@ export const sinkerFormatTypeEnum = { json: 'json' }; +export const sinkerSqlWriteModeEnum = { + append: 'append', + overwrite: 'overwrite', + ignore: 'ignore', + errorifexists: 'errorIfExists' +}; + +export const sinkerSqlWriteModes = [ + { + key: sinkerSqlWriteModeEnum.append, + name: 'Append', + disabled: false + }, + { + key: sinkerSqlWriteModeEnum.overwrite, + name: 'Overwrite', + disabled: false + }, + { + key: sinkerSqlWriteModeEnum.ignore, + name: 'Ignore', + disabled: false + }, + { + key: sinkerSqlWriteModeEnum.errorIfExists, + name: 'Error if exists', + disabled: false + } +]; + export const ruleTypeEnum = { tag: 'tag' }; @@ -112,6 +166,11 @@ export const inputModes = [ key: inputModeEnum.streaming, name: 'Streaming', disabled: false + }, + { + key: inputModeEnum.batching, + name: 'Batching', + disabled: false } ]; @@ -143,6 +202,30 @@ export const inputTypes = [ } ]; +export const inputTypesBatching = [ + { + key: inputTypeEnum.blob, + name: 'Azure Blob', + disabled: false + } +]; + +export const inputCompressionTypes = [ + { + key: inputCompressionTypeEnum.none, + name: 'None', + disabled: false + } +]; + +export const inputFormatTypes = [ + { + key: inputFormatTypeEnum.json, + name: 'JSON', + disabled: false + } +]; + export const watermarkUnits = [ { key: watermarkUnitEnum.second, @@ -229,6 +312,11 @@ export const outputSinkerTypes = [ name: 'Event Hub', disabled: false }, + { + key: sinkerTypeEnum.sql, + name: 'Azure SQL Database', + disabled: false + }, { key: sinkerTypeEnum.local, name: 'Local', @@ -236,6 +324,37 @@ export const outputSinkerTypes = [ } ]; +export const batchTypes = [ + { + key: batchTypeEnum.recurring, + name: 'Recurring', + disabled: false + }, + { + key: batchTypeEnum.oneTime, + name: 'One Time', + disabled: false + } +]; + +export const batchIntervalTypes = [ + { + key: batchIntervalTypeEnum.day, + name: 'Day', + disabled: false + }, + { + key: batchIntervalTypeEnum.hour, + name: 'Hour', + disabled: false + }, + { + key: batchIntervalTypeEnum.min, + name: 'Min', + disabled: false + } +]; + export const sinkerCompressionTypes = [ { key: sinkerCompressionTypeEnum.none, @@ -428,6 +547,18 @@ export const severityTypes = [ } ]; +export function getDefaultBatchInputSettings() { + return { + type: inputTypeEnum.blob, + properties: { + connection: '', + path: '', + formatType: inputFormatTypeEnum.json, + compressionType: inputCompressionTypeEnum.none + } + }; +} + export function getDefaultReferenceDataSettings(type) { if (type === referenceDataTypeEnum.csv) { return { @@ -491,6 +622,44 @@ export function getMetricSinker() { }; } +export function getDefaultBatchSettings(type) { + if (type === batchTypeEnum.oneTime) { + return { + id: '', + type: type, + disabled: false, + properties: { + interval: '1', + intervalType: 'day', + delay: '0', + delayType: 'day', + window: '1', + windowType: 'day', + startTime: '', + endTime: '', + lastProcessedTime: '' + } + }; + } else { + return { + id: '', + type: type, + disabled: false, + properties: { + interval: '1', + intervalType: 'day', + delay: '0', + delayType: 'day', + window: '1', + windowType: 'day', + startTime: new Date(), + endTime: '', + lastProcessedTime: '' + } + }; + } +} + export function getDefaultSinkerSettings(type, owner) { if (type === sinkerTypeEnum.cosmosdb) { return { @@ -538,6 +707,17 @@ export function getDefaultSinkerSettings(type, owner) { compressionType: sinkerCompressionTypeEnum.none } }; + } else if (type === sinkerTypeEnum.sql) { + return { + id: '', + type: type, + properties: { + connectionString: '', + tableName: '', + writeMode: sinkerSqlWriteModeEnum.append, + useBulkInsert: false + } + }; } else { return { id: '', @@ -625,29 +805,6 @@ export function getDefaultInput(enableLocalOneBox) { } } -export function getDefaultQuery(enableLocalOneBox) { - if (enableLocalOneBox) { - return defaultQueryLocal; - } else { - return defaultQuery; - } -} - -//User can use the sample or tutorial or intellisense to have starting query. Below allows default to have 5 blank lines. -export const defaultQuery = ` - -`; - -export const defaultQueryLocal = `--DataXQuery-- -events = SELECT MAX(temperature) as maxTemp - FROM - DataXProcessedInput; - -maxTemperature = CreateMetric(events, maxTemp); - -OUTPUT maxTemperature TO Metrics; -`; - export const defaultSchema = `{ "type": "struct", "fields": [ @@ -700,11 +857,21 @@ export const defaultSchemaLocal = `{ } `; +export function getDefaultNormalizationSnippet(inputMode) { + if (inputMode === inputModeEnum.batching) { + return defaultBatchNormalizationSnippet; + } else { + return defaultNormalizationSnippet; + } +} + export const defaultNormalizationSnippet = `SystemProperties AS _SystemProperties Properties AS _Properties Raw.*`; -//Default Flow settings +export const defaultBatchNormalizationSnippet = `Raw.*`; + +// Default Flow settings export const defaultInput = { type: inputTypeEnum.events, mode: inputModeEnum.streaming, diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowReducer.js b/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowReducer.js index 13a97e293..5c169a8ed 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowReducer.js +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowReducer.js @@ -4,27 +4,38 @@ // ********************************************************************* import * as Actions from './flowActions'; import * as Models from './flowModels'; +import { QueryModels, QueryActions } from 'datax-query'; const INITIAL_FLOW_STATE = { // Flow Config name: '', + flowId: '', displayName: Models.getDefaultName(), owner: '', + databricksToken: '', input: Models.defaultInput, referenceData: [], functions: [], - query: Models.defaultQuery, + query: QueryModels.defaultQuery, scale: { jobNumExecutors: '4', - jobExecutorMemory: '1000' + jobExecutorMemory: '1000', + jobDatabricksAutoScale: false, + jobDatabricksMinWorkers: '3', + jobDatabricksMaxWorkers: '8' }, outputs: [Models.getMetricSinker()], outputTemplates: [], rules: [], + batchInputs: [Models.getDefaultBatchInputSettings()], + batchList: [], + // State isNew: false, isDirty: false, + selectedFlowBatchInputIndex: undefined, + selectedBatchIndex: undefined, selectedReferenceDataIndex: undefined, selectedFunctionIndex: undefined, selectedSinkerIndex: undefined, @@ -35,7 +46,8 @@ const INITIAL_FLOW_STATE = { resamplingInputDuration: '15', errorMessage: undefined, warningMessage: undefined, - enableLocalOneBox: false + enableLocalOneBox: false, + isDatabricksSparkType: false }; export default (state = INITIAL_FLOW_STATE, action) => { @@ -45,6 +57,7 @@ export default (state = INITIAL_FLOW_STATE, action) => { return Object.assign({}, INITIAL_FLOW_STATE, flow, { isNew: false, isDirty: false, + selectedFlowBatchInputIndex: flow.batchInputs && flow.batchInputs.length > 0 ? 0 : undefined, selectedReferenceDataIndex: flow.referenceData && flow.referenceData.length > 0 ? 0 : undefined, selectedFunctionIndex: flow.functions && flow.functions.length > 0 ? 0 : undefined, selectedSinkerIndex: flow.outputs && flow.outputs.length > 0 ? 0 : undefined, @@ -59,9 +72,10 @@ export default (state = INITIAL_FLOW_STATE, action) => { isDirty: true, owner: action.payload, selectedSinkerIndex: 0, // new flow by default contains the metric sinker + selectedFlowBatchInputIndex: 0, enableLocalOneBox: state.enableLocalOneBox ? state.enableLocalOneBox : false, input: Models.getDefaultInput(state.enableLocalOneBox), - query: Models.getDefaultQuery(state.enableLocalOneBox) + query: QueryModels.getDefaultQuery(state.enableLocalOneBox) }); case Actions.FLOW_UPDATE_DISPLAY_NAME: @@ -70,6 +84,17 @@ export default (state = INITIAL_FLOW_STATE, action) => { displayName: action.payload }); + case Actions.FLOW_UPDATE_DATABRICKSTOKEN: + return Object.assign({}, state, { + isDirty: true, + databricksToken: action.payload + }); + + case Actions.FLOW_UPDATE_ISDATABRICKSSPARKTYPE: + return Object.assign({}, state, { + isDatabricksSparkType: action.payload + }); + case Actions.FLOW_UPDATE_OWNER: return Object.assign({}, state, { owner: action.payload }); @@ -79,14 +104,17 @@ export default (state = INITIAL_FLOW_STATE, action) => { input: action.payload }); - case Actions.FLOW_UPDATE_SAMPLING_INPUT_DURATION: + case Actions.FLOW_UPDATE_BATCH_INPUT: + let updatedBatchInputs = [...state.batchInputs]; + updatedBatchInputs[action.index] = action.payload; return Object.assign({}, state, { - samplingInputDuration: action.duration + isDirty: true, + batchInputs: updatedBatchInputs }); - case Actions.FLOW_UPDATE_RESAMPLING_INPUT_DURATION: + case Actions.FLOW_UPDATE_SAMPLING_INPUT_DURATION: return Object.assign({}, state, { - resamplingInputDuration: action.duration + samplingInputDuration: action.duration }); case Actions.FLOW_UPDATE_REFERENCE_DATA_LIST: @@ -179,12 +207,6 @@ export default (state = INITIAL_FLOW_STATE, action) => { case Actions.FLOW_UPDATE_SELECTED_FUNCTION_INDEX: return Object.assign({}, state, { selectedFunctionIndex: action.payload }); - case Actions.FLOW_UPDATE_QUERY: - return Object.assign({}, state, { - isDirty: true, - query: action.payload - }); - case Actions.FLOW_UPDATE_SCALE: return Object.assign({}, state, { isDirty: true, @@ -297,6 +319,46 @@ export default (state = INITIAL_FLOW_STATE, action) => { enableLocalOneBox: action.payload }); + // Batch + case Actions.FLOW_NEW_BATCH: + const batchList = [...state.batchList, Models.getDefaultBatchSettings(action.payload)]; + return Object.assign({}, state, { + isDirty: true, + batchList: batchList, + selectedBatchIndex: batchList.length - 1 + }); + + case Actions.FLOW_DELETE_BATCH: + const deleteBatchIndex = action.index; + let batchListAfterDelete = [...state.batchList]; + batchListAfterDelete.splice(deleteBatchIndex, 1); + + let nextBatchIndex; + if (batchListAfterDelete.length > deleteBatchIndex) { + nextBatchIndex = deleteBatchIndex; + } else if (batchListAfterDelete.length > 0) { + nextBatchIndex = deleteBatchIndex - 1; + } else { + nextBatchIndex = undefined; + } + + return Object.assign({}, state, { + isDirty: true, + batchList: batchListAfterDelete, + selectedBatchIndex: nextBatchIndex + }); + + case Actions.FLOW_UPDATE_SELECTED_BATCH_INDEX: + return Object.assign({}, state, { selectedBatchIndex: action.payload }); + + case Actions.FLOW_UPDATE_BATCHLIST: + let updatedBatchList = [...state.batchList]; + updatedBatchList[action.index] = action.payload; + return Object.assign({}, state, { + isDirty: true, + batchList: updatedBatchList + }); + default: return state; } diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowSelectors.js b/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowSelectors.js index 7ec1cd7a5..e6726ffdb 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowSelectors.js +++ b/Website/Packages/datax-pipeline/src/modules/flowDefinition/flowSelectors.js @@ -5,6 +5,10 @@ import { createSelector } from 'reselect'; import * as Helpers from './flowHelpers'; import * as Models from './flowModels'; +import { CommonHelpers } from 'datax-common'; +import { QuerySelectors } from 'datax-query'; + +const secretScopePrefix = 'secretscope://' // Settings - Flow export const getFlow = state => state.flow; @@ -26,6 +30,16 @@ export const getFlowDisplayName = createSelector( flow => flow.displayName ); +export const getFlowDatabricksToken = createSelector( + getFlow, + flow => flow.databricksToken +); + +export const getFlowIsDatabricksSparkType = createSelector( + getFlow, + flow => flow.isDatabricksSparkType +); + // Settings - Input export const getFlowInput = createSelector( getFlow, @@ -37,6 +51,31 @@ export const getFlowInputProperties = createSelector( input => input.properties ); +export const getFlowBatchInput = createSelector( + getFlow, + flow => flow.batchInputs +); + +export const getSelectedBatchInputIndex = createSelector( + getFlow, + flow => flow.selectedFlowBatchInputIndex +); + +export const getSelectedBatchInput = createSelector( + getFlowBatchInput, + getSelectedBatchInputIndex, + selectedFlowBatchInput +); + +export const getSelectedBatchInputProperties = createSelector( + getSelectedBatchInput, + batch => batch.properties +); + +function selectedFlowBatchInput(batchInputs, selectedIndex) { + return selectedIndex !== undefined && selectedIndex < batchInputs.length ? batchInputs[selectedIndex] : undefined; +} + // Settings - Reference Data export const getFlowReferenceData = createSelector( getFlow, @@ -89,12 +128,6 @@ function selectedFunction(functions, selectedIndex) { return selectedIndex !== undefined && selectedIndex < functions.length ? functions[selectedIndex] : undefined; } -// Settings - Query -export const getFlowQuery = createSelector( - getFlow, - flow => flow.query -); - // Settings - Scale export const getFlowScale = createSelector( getFlow, @@ -107,6 +140,32 @@ export const getFlowOutputs = createSelector( flow => flow.outputs ); +// Settings - Batch +export const getFlowBatchList = createSelector( + getFlow, + flow => flow.batchList +); + +export const getSelectedBatchIndex = createSelector( + getFlow, + flow => flow.selectedBatchIndex +); + +export const getSelectedBatch = createSelector( + getFlowBatchList, + getSelectedBatchIndex, + selectedBatch +); + +export const getSelectedBatchProperties = createSelector( + getSelectedBatch, + batch => batch.properties +); + +function selectedBatch(batchList, selectedIndex) { + return selectedIndex !== undefined && selectedIndex < batchList.length ? batchList[selectedIndex] : undefined; +} + export const getSelectedSinkerIndex = createSelector( getFlow, flow => flow.selectedSinkerIndex @@ -183,64 +242,100 @@ export const getEnableLocalOneBox = createSelector( // Validation - Info export const validateFlowInfo = createSelector( getFlowDisplayName, + getFlowDatabricksToken, + getFlowIsDatabricksSparkType, validateInfo ); -function validateInfo(displayName) { - return displayName && displayName.trim() !== ''; +function validateInfo(displayName, databricksToken, isDatabricksSparkType) { + if (isDatabricksSparkType) { + return displayName && displayName.trim() !== '' && databricksToken && databricksToken.startsWith(secretScopePrefix); + } else { + return displayName && displayName.trim() !== '' + } } // Validation - Input export const validateFlowInput = createSelector( getFlowInput, + getFlowBatchInput, validateInput ); -function validateInput(input) { +function validateInput(input, batchInputs) { + let validations = []; + if (input.mode === Models.inputModeEnum.streaming) { + return validateInputStreaming(input); + } else if (input.mode === Models.inputModeEnum.batching) { + return validateInputBatch(batchInputs); + } else { + validations.push(false); + } + + return validations.every(value => value); +} + +function validateInputStreaming(input) { let validations = []; validations.push(input && input.properties); - if (input.mode === Models.inputModeEnum.streaming) { - if (input.type === Models.inputTypeEnum.events) { - validations.push(input.properties.inputEventhubConnection.trim() !== ''); - validations.push(Helpers.isValidNumberAboveZero(input.properties.windowDuration)); - validations.push( - input.properties.watermarkValue.trim() !== '' && Helpers.isValidNumberAboveOrEqualZero(input.properties.watermarkValue) - ); - validations.push(Helpers.isValidNumberAboveZero(input.properties.maxRate)); - validations.push(Helpers.isValidJson(input.properties.inputSchemaFile)); - } else if (input.type === Models.inputTypeEnum.iothub) { - validations.push(input.properties.inputEventhubName.trim() !== ''); - validations.push(input.properties.inputEventhubConnection.trim() !== ''); - validations.push(Helpers.isValidNumberAboveZero(input.properties.windowDuration)); - validations.push( - input.properties.watermarkValue.trim() !== '' && Helpers.isValidNumberAboveOrEqualZero(input.properties.watermarkValue) - ); - validations.push(Helpers.isValidNumberAboveZero(input.properties.maxRate)); - validations.push(Helpers.isValidJson(input.properties.inputSchemaFile)); - } else if (input.type === Models.inputTypeEnum.kafkaeventhub || input.type === Models.inputTypeEnum.kafka) { - validations.push(input.properties.inputEventhubName.trim() !== ''); - validations.push(input.properties.inputEventhubConnection.trim() !== ''); - validations.push(Helpers.isValidNumberAboveZero(input.properties.windowDuration)); - validations.push( - input.properties.watermarkValue.trim() !== '' && Helpers.isValidNumberAboveOrEqualZero(input.properties.watermarkValue) - ); - validations.push(Helpers.isValidNumberAboveZero(input.properties.maxRate)); - validations.push(Helpers.isValidJson(input.properties.inputSchemaFile)); - } else if (input.type === Models.inputTypeEnum.local) { - validations.push(Helpers.isValidNumberAboveZero(input.properties.windowDuration)); - validations.push( - input.properties.watermarkValue.trim() !== '' && Helpers.isValidNumberAboveOrEqualZero(input.properties.watermarkValue) - ); - validations.push(Helpers.isValidNumberAboveZero(input.properties.maxRate)); - validations.push(Helpers.isValidJson(input.properties.inputSchemaFile)); - } else { - validation.push(false); - } + if (input.type === Models.inputTypeEnum.events) { + validations.push(input.properties.inputEventhubConnection.trim() !== ''); + validations.push(CommonHelpers.isValidNumberAboveZero(input.properties.windowDuration)); + validations.push( + input.properties.watermarkValue.trim() !== '' && Helpers.isValidNumberAboveOrEqualZero(input.properties.watermarkValue) + ); + validations.push(CommonHelpers.isValidNumberAboveZero(input.properties.maxRate)); + validations.push(Helpers.isValidJson(input.properties.inputSchemaFile)); + } else if (input.type === Models.inputTypeEnum.iothub) { + validations.push(input.properties.inputEventhubName.trim() !== ''); + validations.push(input.properties.inputEventhubConnection.trim() !== ''); + validations.push(CommonHelpers.isValidNumberAboveZero(input.properties.windowDuration)); + validations.push( + input.properties.watermarkValue.trim() !== '' && Helpers.isValidNumberAboveOrEqualZero(input.properties.watermarkValue) + ); + validations.push(CommonHelpers.isValidNumberAboveZero(input.properties.maxRate)); + validations.push(Helpers.isValidJson(input.properties.inputSchemaFile)); + } else if (input.type === Models.inputTypeEnum.kafkaeventhub || input.type === Models.inputTypeEnum.kafka) { + validations.push(input.properties.inputEventhubName.trim() !== ''); + validations.push(input.properties.inputEventhubConnection.trim() !== ''); + validations.push(CommonHelpers.isValidNumberAboveZero(input.properties.windowDuration)); + validations.push( + input.properties.watermarkValue.trim() !== '' && Helpers.isValidNumberAboveOrEqualZero(input.properties.watermarkValue) + ); + validations.push(CommonHelpers.isValidNumberAboveZero(input.properties.maxRate)); + validations.push(Helpers.isValidJson(input.properties.inputSchemaFile)); + } else if (input.type === Models.inputTypeEnum.local) { + validations.push(CommonHelpers.isValidNumberAboveZero(input.properties.windowDuration)); + validations.push( + input.properties.watermarkValue.trim() !== '' && Helpers.isValidNumberAboveOrEqualZero(input.properties.watermarkValue) + ); + validations.push(CommonHelpers.isValidNumberAboveZero(input.properties.maxRate)); + validations.push(Helpers.isValidJson(input.properties.inputSchemaFile)); + } else { + validation.push(false); + } + + return validations.every(value => value); +} + +function validateInputBatch(batchInputs) { + return batchInputs && batchInputs.every(isBatchInputSettingsComplete); +} + +function isBatchInputSettingsComplete(batchInput) { + let validations = []; + validations.push(batchInput && batchInput.properties); + + if (batchInput.type === Models.inputTypeEnum.blob) { + validations.push(batchInput.properties.connection && batchInput.properties.connection.trim() !== ''); + validations.push(batchInput.properties.path && batchInput.properties.path.trim() !== ''); + validations.push(batchInput.properties.formatType && batchInput.properties.formatType.trim() !== ''); + validations.push(batchInput.properties.compressionType && batchInput.properties.compressionType.trim() !== ''); } else { - // future support validation.push(false); } + return validations.every(value => value); } @@ -318,17 +413,6 @@ function isFunctionSettingsComplete(functionItem) { return validations.every(value => value); } -// Validation - Query -export const validateFlowQuery = createSelector( - getFlowQuery, - validateQuery -); - -function validateQuery(query) { - //removing validation; codegen will add OUTPUTs for alerts. Blank query is valid. - return query || query.trim() === ''; -} - // Validation - Outputs export const validateFlowOutputs = createSelector( getFlowOutputs, @@ -362,6 +446,11 @@ function isSinkerSettingsComplete(sinker) { validations.push(sinker.properties.blobPartitionFormat && sinker.properties.blobPartitionFormat.trim() !== ''); break; + case Models.sinkerTypeEnum.sql: + validations.push(sinker.properties.connectionString && sinker.properties.connectionString.trim() !== ''); + validations.push(sinker.properties.tableName && sinker.properties.connectionString.trim() !== ''); + break; + case Models.sinkerTypeEnum.metric: // no additional validation needed, this is a system provided sinker type break; @@ -452,7 +541,63 @@ export const validateFlowScale = createSelector( ); function validateScale(scale) { - return scale && Helpers.isValidNumberAboveZero(scale.jobNumExecutors) && Helpers.isValidNumberAboveZero(scale.jobExecutorMemory); + return ( + scale && + CommonHelpers.isValidNumberAboveZero(scale.jobNumExecutors) && + CommonHelpers.isValidNumberAboveZero(scale.jobExecutorMemory) + ); +} + +// Validation - Schedule +export const validateFlowSchedule = createSelector( + getFlowInput, + getFlowBatchList, + validateSchedule +); + +function validateSchedule(flowInput, batchList) { + return ( + (flowInput && flowInput.mode === Models.inputModeEnum.streaming) || + (batchList && batchList.length > 0 && batchList.every(isBatchListSettingsComplete)) + ); +} + +function isBatchListSettingsComplete(batch) { + let validations = []; + validations.push(batch && batch.properties); + validations.push(Helpers.isNumberAndStringOnly(batch.id)); + + switch (batch.type) { + case Models.batchTypeEnum.recurring: + validations.push(batch.properties.interval && batch.properties.interval.trim() !== ''); + validations.push(batch.properties.intervalType && batch.properties.intervalType.trim() !== ''); + validations.push(batch.properties.delay && batch.properties.delay.trim() !== ''); + validations.push(batch.properties.delayType && batch.properties.delayType.trim() !== ''); + validations.push(batch.properties.window && batch.properties.window.trim() !== ''); + validations.push(batch.properties.windowType && batch.properties.windowType.trim() !== ''); + validations.push(batch.properties.startTime && batch.properties.startTime !== ''); + validations.push(CommonHelpers.isValidNumberAboveZero(batch.properties.interval)); + validations.push(Helpers.isValidNumberAboveOrEqualZero(batch.properties.delay)); + validations.push(CommonHelpers.isValidNumberAboveZero(batch.properties.window)); + break; + case Models.batchTypeEnum.oneTime: + validations.push(batch.properties.interval && batch.properties.interval.trim() !== ''); + validations.push(batch.properties.intervalType && batch.properties.intervalType.trim() !== ''); + validations.push(batch.properties.delay && batch.properties.delay == 0); + validations.push(batch.properties.delayType && batch.properties.delayType.trim() !== ''); + validations.push(batch.properties.window && batch.properties.window.trim() !== ''); + validations.push(batch.properties.windowType && batch.properties.windowType.trim() !== ''); + validations.push(batch.properties.startTime && batch.properties.startTime !== ''); + validations.push(batch.properties.endTime && batch.properties.endTime !== ''); + validations.push(CommonHelpers.isValidNumberAboveZero(batch.properties.interval)); + validations.push(CommonHelpers.isValidNumberAboveZero(batch.properties.window)); + break; + + default: + validations.push(false); + break; + } + return validations.every(value => value); } // Validation - Flow @@ -460,10 +605,11 @@ export const validateFlow = createSelector( validateFlowInfo, validateFlowInput, validateFlowFunctions, - validateFlowQuery, + QuerySelectors.validateQueryTab, validateFlowOutputs, validateFlowOutputTemplates, validateFlowRules, validateFlowScale, + validateFlowSchedule, (...selectors) => selectors.every(value => value) ); diff --git a/Website/Packages/datax-pipeline/src/modules/flowList/api.js b/Website/Packages/datax-pipeline/src/modules/flowList/api.js index 9646cdffd..fda252cca 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowList/api.js +++ b/Website/Packages/datax-pipeline/src/modules/flowList/api.js @@ -2,8 +2,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License // ********************************************************************* -import { serviceGetApi } from 'datax-common'; -import { Constants } from '../../common/apiConstants'; +import { serviceGetApi, Constants } from 'datax-common'; export const getFlowsList = () => serviceGetApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.flow, 'flow/getall/min'); diff --git a/Website/Packages/datax-pipeline/src/styles/styles.css b/Website/Packages/datax-pipeline/src/styles/styles.css index 33032770e..a3b4485d9 100644 --- a/Website/Packages/datax-pipeline/src/styles/styles.css +++ b/Website/Packages/datax-pipeline/src/styles/styles.css @@ -117,23 +117,13 @@ background-color: #666666 !important; } -/* Query Pane styles */ -.query-pane-button { - height: 27px !important; - padding-left: 5px !important; - padding-right: 5px !important; - min-width: 1px !important; - } - -.ms-IconExample:hover { - cursor: pointer !important; -} - -.query-pane-TextField{ - height: 27px !important; - width: 50px !important; -} - -.query-pane-TextField .ms-TextField-fieldGroup { - height: 27px !important; +.react-datetime-picker__wrapper { + display: flex; + border: thin solid #d0d0d0 !important; + width: 100%; + height: 32px; + padding-right: 12px; + padding-left: 12px; + font-size: 14px; + background-color: white; } \ No newline at end of file diff --git a/Website/Packages/datax-pipeline/webpack.config.js b/Website/Packages/datax-pipeline/webpack.config.js index ad0a374f9..f37913c9f 100644 --- a/Website/Packages/datax-pipeline/webpack.config.js +++ b/Website/Packages/datax-pipeline/webpack.config.js @@ -6,7 +6,6 @@ const webpack = require('webpack'); const path = require('path'); const pkg = require('./package.json'); const MiniCssExtractPlugin = require('mini-css-extract-plugin'); -const MonacoWebpackPlugin = require('monaco-editor-webpack-plugin'); const TerserPlugin = require('terser-webpack-plugin'); const libraryName = pkg.name; @@ -30,8 +29,11 @@ function configure(env, argv) { // https://webpack.js.org/configuration/resolve/ resolve: { extensions: ['.js', '.jsx'], - modules: [path.resolve(__dirname, 'src'), 'node_modules'] - }, + modules: [path.resolve(__dirname, 'src'), 'node_modules'], + alias: { 'react': path.resolve(__dirname, './node_modules/', 'react'), + 'react-dom': path.resolve('./node_modules/react-dom') + } + }, // https://webpack.js.org/configuration/externals/ externals: { react: 'umd react', @@ -53,10 +55,6 @@ function configure(env, argv) { '@babel/preset-env', // https://babeljs.io/docs/en/babel-preset-react '@babel/preset-react' - ], - plugins: [ - //https://babeljs.io/docs/en/babel-plugin-syntax-dynamic-import - '@babel/plugin-syntax-dynamic-import' ] } } @@ -94,17 +92,9 @@ function configure(env, argv) { }, //https://webpack.js.org/configuration/plugins/ plugins: [ - // https://github.com/Microsoft/monaco-editor-webpack-plugin - new MonacoWebpackPlugin({ - languages: ['sql'] - }), // https://github.com/webpack-contrib/mini-css-extract-plugin new MiniCssExtractPlugin({ filename: 'css/[name].css' - }), - // https://webpack.js.org/plugins/limit-chunk-count-plugin/ - new webpack.optimize.LimitChunkCountPlugin({ - maxChunks: 1 }) ] }; diff --git a/Website/Packages/datax-query/.gitignore b/Website/Packages/datax-query/.gitignore new file mode 100644 index 000000000..16acd49d7 --- /dev/null +++ b/Website/Packages/datax-query/.gitignore @@ -0,0 +1,3 @@ +node_modules +dist +package-lock.json diff --git a/Website/Packages/datax-query/.prettierrc b/Website/Packages/datax-query/.prettierrc new file mode 100644 index 000000000..ef0d45c28 --- /dev/null +++ b/Website/Packages/datax-query/.prettierrc @@ -0,0 +1,12 @@ +{ + "printWidth": 140, + "tabWidth": 4, + "useTabs": false, + "semi": true, + "singleQuote": true, + "jsxSingleQuote": false, + "trailingComma": "none", + "bracketSpacing": true, + "jsxBracketSameLine": false, + "arrowParens": "avoid" +} diff --git a/Website/Packages/datax-query/.vscode/settings.json b/Website/Packages/datax-query/.vscode/settings.json new file mode 100644 index 000000000..43bb7ed7e --- /dev/null +++ b/Website/Packages/datax-query/.vscode/settings.json @@ -0,0 +1,13 @@ +{ + "[javascript]": { + "editor.formatOnSave": true + }, + "[javascriptreact]": { + "editor.formatOnSave": true + }, + "search.exclude": { + "**/node_modules": true, + "**/bower_components": true, + "**/dist": true + } +} diff --git a/Website/Packages/datax-query/README.md b/Website/Packages/datax-query/README.md new file mode 100644 index 000000000..acb5d1ebd --- /dev/null +++ b/Website/Packages/datax-query/README.md @@ -0,0 +1,30 @@ +# datax-query +Contains query features representing a streaming/batching data pipeline configuration management experience. +This package is used to add additional features to the Data Accelerator website. + +### Install Package to Accelerator Website or another package +1. At root folder of your consumer project, run ```npm install datax-query@1.1.1``` (or whatever version you want). +This will install the package under the ```node_modules``` folder and automatically update the ```package.json``` file of your project. +You can also manually input it into the ```package.json``` yourself and then run ```npm install```. + +### Quick start to developing this package +1. Run ```npm install``` to install all dependency package of this NPM package. + +2. Run ```npm run dev``` to build non-optimized bundles. While the packages tend to be larger in size and slow down your web experience, you benefit +this by getting a better development experience when debugging the sources on the browser of your choice. + +3. When you are done developing your package, increment the version number of your NPM package in the ```package.json``` file. + +4. Run ```npm run build``` to build optimized bundles (obfuscated, minified and other compiler optimizations) leading to smaller output sizes and +faster performance for production consumption. + +5. Run ```npm publish``` + +### Tips and Tricks +1. For your website, run ```npm run devwatch``` which will put your website into listening mode for file changes. Every time a file that the website +depends on under its folder and ```node_modules``` dependency folder changes, it will automatically re-compile. + +2. Run ```npm run devpatch``` to build development bundles and this command will automatically execute a batch script to xcopy the built bundles +to our website's ```node_modules``` folder. This will cause your website to recompile itself to pick up the changes. +This ```localdevpatch.bat``` script assumes that the website GIT repo and this packages GIT repo share the same parent folder. +If this is not the case, please change the paths of the script locally on your computer. diff --git a/Website/Packages/datax-query/localdevpatch.bat b/Website/Packages/datax-query/localdevpatch.bat new file mode 100644 index 000000000..054359879 --- /dev/null +++ b/Website/Packages/datax-query/localdevpatch.bat @@ -0,0 +1,14 @@ +@echo on +REM Local Development Patch + +@echo off +REM Patch this package's dist bits to Website's node_module/this_package_name/dist folder +REM Change destination path as needed if website location is different + +@echo on +REM Patching dist folder to pipeline +xcopy dist ..\datax-pipeline\node_modules\datax-query\dist /i /s /y + +@echo off +REM Patch all other dist folders that you want to patch here +REM For example, you may want to patch the feature packages that uses this package \ No newline at end of file diff --git a/Website/Packages/datax-query/package.json b/Website/Packages/datax-query/package.json new file mode 100644 index 000000000..569686289 --- /dev/null +++ b/Website/Packages/datax-query/package.json @@ -0,0 +1,66 @@ +{ + "name": "datax-query", + "version": "1.3.0", + "description": "Query features", + "author": "Microsoft", + "license": "MIT", + "main": "./dist/index.js", + "files": [ + "dist" + ], + "engines": { + "node": "10.6.0" + }, + "repository": { + "type": "git", + "url": "https://github.com/Microsoft/data-accelerator" + }, + "scripts": { + "test": "echo \"Error: no test specified\" && exit 1", + "build": "rimraf dist && webpack --mode production --config webpack.config.js --bail --progress --display-modules --display-error-details --colors --display-chunks --display-reasons", + "dev": "rimraf dist && webpack --mode development --config webpack.config.js --bail --progress --display-modules --display-error-details --colors --display-chunks --display-reasons", + "devpatch": "rimraf dist && webpack --mode development --config webpack.config.js --bail --progress --display-modules --display-error-details --colors --display-chunks --display-reasons && localdevpatch.bat", + "devwatch": "webpack --mode development --config webpack.config.js --bail --progress --display-modules --display-error-details --colors --display-chunks --display-reasons --watch" + }, + "dependencies": {}, + "devDependencies": { + "@babel/core": "7.1.6", + "@babel/plugin-syntax-dynamic-import": "7.0.0", + "@babel/preset-env": "7.1.6", + "@babel/preset-react": "7.0.0", + "babel-loader": "8.0.4", + "css-loader": "1.0.1", + "file-loader": "2.0.0", + "mini-css-extract-plugin": "0.4.4", + "datax-common": "1.3.0", + "office-ui-fabric-react": "6.111.2", + "q": "1.5.1", + "promise-polyfill": "8.1.0", + "prop-types": "15.6.2", + "radium": "0.25.0", + "react": "16.6.3", + "react-ace": "6.2.0", + "react-dom": "16.6.3", + "react-redux": "5.1.1", + "jsoneditor": "5.26.2", + "jsoneditor-react": "1.0.0", + "monaco-editor-webpack-plugin": "1.6.0", + "react-monaco-editor": "0.22.0", + "react-router": "4.3.1", + "react-router-dom": "4.3.1", + "react-splitter-layout": "3.0.1", + "redux": "4.0.1", + "redux-thunk": "2.3.0", + "reselect": "4.0.0", + "rimraf": "2.6.2", + "terser-webpack-plugin": "1.1.0", + "url-loader": "1.1.2", + "webpack": "4.26.0", + "webpack-cli": "3.1.2" + }, + "peerDependencies": { + "q": "1.5.1", + "react": "16.6.3", + "react-dom": "16.6.3" + } +} diff --git a/Website/Packages/datax-query/src/index.js b/Website/Packages/datax-query/src/index.js new file mode 100644 index 000000000..3e7573e77 --- /dev/null +++ b/Website/Packages/datax-query/src/index.js @@ -0,0 +1,14 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +import 'jsoneditor-react/es/editor.min.css'; +import 'monaco-editor/min/vs/editor/editor.main.css'; +import './styles/styles.css'; +/** + * Export all modules to expose out of this package library + */ +export * from './modules'; +export { JsonEditor } from 'jsoneditor-react'; +export { default as MonacoEditorControl} from './modules/query/components/monacoeditorcontrol' \ No newline at end of file diff --git a/Website/Packages/datax-query/src/modules/index.js b/Website/Packages/datax-query/src/modules/index.js new file mode 100644 index 000000000..d80bb729a --- /dev/null +++ b/Website/Packages/datax-query/src/modules/index.js @@ -0,0 +1,31 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +/** + * Modules - Specify here all the common modules you want to expose externally + */ +// Query +import * as KernelActions from './query/kernelActions'; +import * as KernelSelectors from './query/kernelSelectors'; +import * as LayoutActions from './query/layoutActions'; +import * as LayoutSelectors from './query/layoutSelectors'; +import * as QueryActions from './query/queryActions'; +import * as QuerySelectors from './query/querySelectors'; +import * as QueryModels from './query/queryModels'; +import * as QueryApi from './query/api'; +export { + QueryApi, + KernelActions, + KernelSelectors, + QueryActions, + QueryModels, + QuerySelectors, + LayoutActions, + LayoutSelectors +}; +export { default as layoutReducer } from './query/layoutReducer'; +export { default as queryReducer } from './query/queryReducer'; +export { default as kernelReducer } from './query/kernelReducer'; +export * from './query/components'; \ No newline at end of file diff --git a/Website/Packages/datax-query/src/modules/query/api.js b/Website/Packages/datax-query/src/modules/query/api.js new file mode 100644 index 000000000..faf4079b1 --- /dev/null +++ b/Website/Packages/datax-query/src/modules/query/api.js @@ -0,0 +1,92 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +import { Constants, servicePostApi } from 'datax-common'; + +export const getTableSchemas = queryMetadata => + servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.flow, 'userqueries/schema', { + name: queryMetadata.name, + displayName: queryMetadata.displayName, + query: queryMetadata.query, + inputSchema: queryMetadata.inputSchema, + rules: queryMetadata.rules, + outputTemplates: queryMetadata.outputTemplates + }); + +export const getCodeGenQuery = queryMetadata => + servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.flow, 'userqueries/codegen', { + name: queryMetadata.name, + displayName: queryMetadata.displayName, + query: queryMetadata.query, + rules: queryMetadata.rules, + outputTemplates: queryMetadata.outputTemplates + }); + +// Interactive Query +export const getDiagnosticKernel = queryMetadata => + servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.interactiveQuery, 'kernel', { + name: queryMetadata.name, + displayName: queryMetadata.displayName, + userName: queryMetadata.userName, + inputSchema: queryMetadata.inputSchema, + normalizationSnippet: queryMetadata.normalizationSnippet, + refData: queryMetadata.refData, + functions: queryMetadata.functions + }); + +export const refreshDiagnosticKernel = (queryMetadata, kernelId) => + servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.interactiveQuery, 'kernel/refresh', { + kernelId: kernelId, + userName: queryMetadata.userName, + name: queryMetadata.name, + displayName: queryMetadata.displayName, + inputSchema: queryMetadata.inputSchema, + normalizationSnippet: queryMetadata.normalizationSnippet, + refData: queryMetadata.refData, + functions: queryMetadata.functions + }); + +export const deleteAllKernels = flowName => + servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.interactiveQuery, 'kernels/deleteall', flowName); + +export const deleteDiagnosticKernelOnUnload = (kernelId, flowName) => + servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.interactiveQuery, 'kernel/delete', { + kernelId: kernelId, + name: flowName + }); + +export const deleteDiagnosticKernel = deleteDiagnosticKernelOnUnload; + +export const executeQuery = (queryMetadata, selectedQuery, kernelId) => + servicePostApi(Constants.serviceRouteApi, Constants.serviceApplication, Constants.services.interactiveQuery, 'kernel/executequery', { + name: queryMetadata.name, + displayName: queryMetadata.displayName, + query: selectedQuery, + kernelId: kernelId, + rules: queryMetadata.rules, + outputTemplates: queryMetadata.outputTemplates + }); + +// Live Data +export const resampleInput = (queryMetadata, kernelId) => + servicePostApi( + Constants.serviceRouteApi, + Constants.serviceApplication, + Constants.services.liveData, + 'inputdata/refreshsampleandkernel', + { + name: queryMetadata.name, + displayName: queryMetadata.displayName, + userName: queryMetadata.userName, + kernelId: kernelId, + inputSchema: queryMetadata.inputSchema, + normalizationSnippet: queryMetadata.normalizationSnippet, + eventhubConnectionString: queryMetadata.eventhubConnection, + inputSubscriptionId: queryMetadata.inputSubscriptionId, + inputResourceGroup: queryMetadata.inputResourceGroup, + eventHubNames: queryMetadata.eventhubNames, + inputType: queryMetadata.inputType, + seconds: queryMetadata.seconds + } + ); diff --git a/Website/Packages/datax-query/src/modules/query/components/index.js b/Website/Packages/datax-query/src/modules/query/components/index.js new file mode 100644 index 000000000..23b236276 --- /dev/null +++ b/Website/Packages/datax-query/src/modules/query/components/index.js @@ -0,0 +1,6 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +export { default as QuerySettingsContent } from './querySettingsContent'; +export { default as SideToolBar } from './sideToolBar'; \ No newline at end of file diff --git a/Website/Packages/datax-query/src/modules/query/components/monacoeditorcontrol.jsx b/Website/Packages/datax-query/src/modules/query/components/monacoeditorcontrol.jsx new file mode 100644 index 000000000..3056d426c --- /dev/null +++ b/Website/Packages/datax-query/src/modules/query/components/monacoeditorcontrol.jsx @@ -0,0 +1,56 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +/** + * Adding control MonacoEditorControl - This can be consumed by the consumers of datax-query without taking any dependency on monaco-editor packages + */ +import React from 'react'; +import Radium from 'radium'; +import PropTypes from 'prop-types'; +import MonacoEditor from 'react-monaco-editor'; + +const MonacoEditorControl = Radium( + class MonacoEditorControl extends React.Component { + constructor(props) { + super(props); + + } + render() { + return ( + + ); + } + } +); + + +// Props +MonacoEditorControl.propTypes = { + name: PropTypes.string.isRequired, + height: PropTypes.string, + width: PropTypes.string, + fontSize: PropTypes.string, + language: PropTypes.string, + theme: PropTypes.string, + value: PropTypes.string, + options: PropTypes.object, + onChange: PropTypes.func, + editorWillMount: PropTypes.func, + editorDidMount: PropTypes.func +}; + +export default MonacoEditorControl; diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/query/querySettingsContent.jsx b/Website/Packages/datax-query/src/modules/query/components/querySettingsContent.jsx similarity index 98% rename from Website/Packages/datax-pipeline/src/modules/flowDefinition/components/query/querySettingsContent.jsx rename to Website/Packages/datax-query/src/modules/query/components/querySettingsContent.jsx index fc7ce3bd8..34ff1af78 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/components/query/querySettingsContent.jsx +++ b/Website/Packages/datax-query/src/modules/query/components/querySettingsContent.jsx @@ -8,15 +8,14 @@ import SideToolBar from './sideToolBar'; import { DefaultButton, Label, TextField } from 'office-ui-fabric-react'; import { Icon } from 'office-ui-fabric-react/lib/Icon'; import brace from 'brace'; -import MonacoEditor from 'react-monaco-editor'; +import MonacoEditorControl from './monacoeditorcontrol'; import 'brace/mode/sql'; import 'brace/theme/xcode'; import 'brace/ext/language_tools'; import { JsonEditor } from 'jsoneditor-react'; import SplitterLayout from 'react-splitter-layout'; -import { Colors, IconButtonStyles, LoadingPanel } from 'datax-common'; -import * as Helpers from '../../flowHelpers'; -import * as Actions from '../../flowActions'; +import { Colors, IconButtonStyles, LoadingPanel, CommonHelpers as Helpers } from 'datax-common'; +import * as Actions from '../queryActions'; const queryExampleWiki = 'https://aka.ms/data-accelerator-query'; @@ -130,7 +129,7 @@ export default class QuerySettingsContent extends React.Component { let editor; if (this.state.showCodeGenQuery) { editor = ( - {return;}} editorWillMount={monaco => this.editorWillMount(monaco)} editorDidMount={editor => this.editorDidMount(editor)} /> ); } else { editor = ( - dispatch => { }); }; -export const getKernel = (flow, version, updateErrorMessage) => (dispatch, getState) => { +export const getKernel = (queryMetadata, version, updateErrorMessage) => (dispatch, getState) => { updateErrorMessage(dispatch, undefined); fetchingKernel(dispatch, true); - return Api.getDiagnosticKernel(flow) + return Api.getDiagnosticKernel(queryMetadata) .then(response => { const kernelId = response.result; const warning = response.message; @@ -48,7 +48,7 @@ export const getKernel = (flow, version, updateErrorMessage) => (dispatch, getSt if (version >= curVersion) { return updateKernel(dispatch, kernelId, version, warning); } else { - return Api.deleteDiagnosticKernel(kernelId); + return Api.deleteDiagnosticKernel(kernelId, queryMetadata.name); } }) .catch(error => { @@ -59,10 +59,10 @@ export const getKernel = (flow, version, updateErrorMessage) => (dispatch, getSt }); }; -export const refreshKernel = (flow, kernelId, version, updateErrorMessage) => (dispatch, getState) => { +export const refreshKernel = (queryMetadata, kernelId, version, updateErrorMessage) => (dispatch, getState) => { updateErrorMessage(dispatch, undefined); fetchingKernel(dispatch, true); - return Api.refreshDiagnosticKernel(flow, kernelId) + return Api.refreshDiagnosticKernel(queryMetadata, kernelId) .then(response => { const kernelId = response.result; const warning = response.message; @@ -72,7 +72,7 @@ export const refreshKernel = (flow, kernelId, version, updateErrorMessage) => (d if (version >= curVersion) { return updateKernel(dispatch, kernelId, version, warning); } else { - return Api.deleteDiagnosticKernel(kernelId); + return Api.deleteDiagnosticKernel(kernelId, queryMetadata.name); } }) .catch(error => { @@ -83,9 +83,9 @@ export const refreshKernel = (flow, kernelId, version, updateErrorMessage) => (d }); }; -export const deleteAllKernels = updateErrorMessage => dispatch => { +export const deleteAllKernels = (updateErrorMessage, flowName) => dispatch => { updateErrorMessage(dispatch, undefined); - return Api.deleteAllKernels() + return Api.deleteAllKernels(flowName) .then(result => { return result; }) @@ -96,8 +96,8 @@ export const deleteAllKernels = updateErrorMessage => dispatch => { }); }; -export const deleteKernel = (kernelId, version) => dispatch => { - return Api.deleteDiagnosticKernel(kernelId) +export const deleteKernel = (kernelId, version, flowName) => dispatch => { + return Api.deleteDiagnosticKernel(kernelId, flowName) .then(result => { return updateKernel(dispatch, '', version, undefined); }) diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/kernelReducer.js b/Website/Packages/datax-query/src/modules/query/kernelReducer.js similarity index 100% rename from Website/Packages/datax-pipeline/src/modules/flowDefinition/kernelReducer.js rename to Website/Packages/datax-query/src/modules/query/kernelReducer.js diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/kernelSelectors.js b/Website/Packages/datax-query/src/modules/query/kernelSelectors.js similarity index 73% rename from Website/Packages/datax-pipeline/src/modules/flowDefinition/kernelSelectors.js rename to Website/Packages/datax-query/src/modules/query/kernelSelectors.js index 03bd2b927..fbf589994 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/kernelSelectors.js +++ b/Website/Packages/datax-query/src/modules/query/kernelSelectors.js @@ -4,19 +4,19 @@ // ********************************************************************* import { createSelector } from 'reselect'; -export const getKernelInfo = state => state.flowKernel; +export const getKernelInfo = state => state.queryKernel; export const getKernelId = createSelector( getKernelInfo, - flowKernel => flowKernel.kernelId + queryKernel => queryKernel.kernelId ); export const getKernelVersion = createSelector( getKernelInfo, - flowKernel => flowKernel.version + queryKernel => queryKernel.version ); export const getFetchingKernel = createSelector( getKernelInfo, - flowKernel => flowKernel.fetchingKernel + queryKernel => queryKernel.fetchingKernel ); diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/layoutActions.js b/Website/Packages/datax-query/src/modules/query/layoutActions.js similarity index 100% rename from Website/Packages/datax-pipeline/src/modules/flowDefinition/layoutActions.js rename to Website/Packages/datax-query/src/modules/query/layoutActions.js diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/layoutReducer.js b/Website/Packages/datax-query/src/modules/query/layoutReducer.js similarity index 100% rename from Website/Packages/datax-pipeline/src/modules/flowDefinition/layoutReducer.js rename to Website/Packages/datax-query/src/modules/query/layoutReducer.js diff --git a/Website/Packages/datax-pipeline/src/modules/flowDefinition/layoutSelectors.js b/Website/Packages/datax-query/src/modules/query/layoutSelectors.js similarity index 72% rename from Website/Packages/datax-pipeline/src/modules/flowDefinition/layoutSelectors.js rename to Website/Packages/datax-query/src/modules/query/layoutSelectors.js index cd5d19651..1b40de1dc 100644 --- a/Website/Packages/datax-pipeline/src/modules/flowDefinition/layoutSelectors.js +++ b/Website/Packages/datax-query/src/modules/query/layoutSelectors.js @@ -4,9 +4,9 @@ // ********************************************************************* import { createSelector } from 'reselect'; -export const getLayoutSettings = state => state.flowLayoutSettings; +export const getLayoutSettings = state => state.queryLayoutSettings; export const getTestQueryOutputPanelVisibility = createSelector( getLayoutSettings, - flowLayoutSettings => flowLayoutSettings.isTestQueryOutputPanelVisible + queryLayoutSettings => queryLayoutSettings.isTestQueryOutputPanelVisible ); diff --git a/Website/Packages/datax-query/src/modules/query/queryActions.js b/Website/Packages/datax-query/src/modules/query/queryActions.js new file mode 100644 index 000000000..11e757d5d --- /dev/null +++ b/Website/Packages/datax-query/src/modules/query/queryActions.js @@ -0,0 +1,119 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +import Q from 'q'; +import { getApiErrorMessage } from 'datax-common'; +import * as Api from './api'; +import * as KernelActions from './kernelActions'; +import * as KernelSelectors from './kernelSelectors'; + +/** + * + * REDUX Action Types + * + */ + +// Init +export const QUERY_INIT = 'QUERY_INIT'; + +// Query +export const QUERY_UPDATE_QUERY = 'QUERY_UPDATE_QUERY'; + + +export const QUERY_UPDATE_RESAMPLING_INPUT_DURATION = 'QUERY_UPDATE_RESAMPLING_INPUT_DURATION'; +export const QUERY_UPDATE_ERROR_MESSAGE = 'QUERY_UPDATE_ERROR_MESSAGE'; +export const QUERY_UPDATE_WARNING_MESSAGE = 'QUERY_UPDATE_WARNING_MESSAGE'; + +export const updateResamplingInputDuration = duration => dispatch => { + return dispatch({ + type: QUERY_UPDATE_RESAMPLING_INPUT_DURATION, + duration: duration + }); +}; + + +export const initQuery = query => dispatch => { + return dispatch({ + type: QUERY_INIT, + payload: query + }); +}; + +// Query Actions +export const updateQuery = query => dispatch => { + return dispatch({ + type: QUERY_UPDATE_QUERY, + payload: query + }); +}; + +export const getTableSchemas = (queryMetadata) => { + return Api.getTableSchemas(queryMetadata).then(tables => { + let tableToSchemaMap = {}; + tables.forEach(table => { + tableToSchemaMap[table.name] = table; + }); + + return tableToSchemaMap; + }); +}; + +export const getCodeGenQuery = (queryMetadata) => { + return Api.getCodeGenQuery(queryMetadata).then(query => { + return query; + }); +}; + +export const executeQuery = (queryMetadata, selectedQuery, kernelId) => { + return dispatch => { + updateErrorMessage(dispatch, undefined); + return Api.executeQuery(queryMetadata, selectedQuery, kernelId) + .then(result => { + return result; + }) + .catch(error => { + const message = getApiErrorMessage(error); + updateErrorMessage(dispatch, message); + return Q.reject({ error: true, message: message }); + }); + }; +}; + +export const resampleInput = (queryMetadata, kernelId, version) => (dispatch, getState) => { + updateErrorMessage(dispatch, undefined); + KernelActions.fetchingKernel(dispatch, true); + return Api.resampleInput(queryMetadata, kernelId) + .then(response => { + const kernelId = response.result; + const warning = response.message; + + const curVersion = KernelSelectors.getKernelVersion(getState()); + + if (version >= curVersion) { + return KernelActions.updateKernel(dispatch, kernelId, version, warning); + } else { + return Api.deleteDiagnosticKernel(kernelId, queryMetadata.name); + } + }) + .catch(error => { + const message = getApiErrorMessage(error); + updateErrorMessage(dispatch, message); + KernelActions.fetchingKernel(dispatch, false); + return Q.reject({ error: true, message: message }); + }); +}; + +export function updateErrorMessage(dispatch, message) { + return dispatch({ + type: QUERY_UPDATE_ERROR_MESSAGE, + message: message + }); +} + +export const updateWarningMessage = message => dispatch => { + return dispatch({ + type: QUERY_UPDATE_WARNING_MESSAGE, + warning: message + }); +}; \ No newline at end of file diff --git a/Website/Packages/datax-query/src/modules/query/queryModels.js b/Website/Packages/datax-query/src/modules/query/queryModels.js new file mode 100644 index 000000000..b729bcfa6 --- /dev/null +++ b/Website/Packages/datax-query/src/modules/query/queryModels.js @@ -0,0 +1,26 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* + +export function getDefaultQuery(enableLocalOneBox) { + if (enableLocalOneBox) { + return defaultQueryLocal; + } else { + return defaultQuery; + } +} +//User can use the sample or tutorial or intellisense to have starting query. Below allows default to have 5 blank lines. +export const defaultQuery = ` + +`; + +export const defaultQueryLocal = `--DataXQuery-- +events = SELECT MAX(temperature) as maxTemp + FROM + DataXProcessedInput; + +maxTemperature = CreateMetric(events, maxTemp); + +OUTPUT maxTemperature TO Metrics; +`; \ No newline at end of file diff --git a/Website/Packages/datax-query/src/modules/query/queryReducer.js b/Website/Packages/datax-query/src/modules/query/queryReducer.js new file mode 100644 index 000000000..8cba3e27e --- /dev/null +++ b/Website/Packages/datax-query/src/modules/query/queryReducer.js @@ -0,0 +1,47 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +import * as Actions from './queryActions'; +import * as Models from './queryModels'; + +const INITIAL_QUERY_STATE = { + query: Models.defaultQuery, + errorMessage: undefined, + warningMessage: undefined, + isDirty: false, + resamplingInputDuration:'15' +}; + +export default (state = INITIAL_QUERY_STATE, action) => { + switch (action.type) { + + case Actions.QUERY_INIT: + const query = action.payload; + return Object.assign({}, INITIAL_QUERY_STATE, query, { + query: query + }); + + + + case Actions.QUERY_UPDATE_RESAMPLING_INPUT_DURATION: + return Object.assign({}, state, { + resamplingInputDuration: action.duration + }); + + case Actions.QUERY_UPDATE_QUERY: + return Object.assign({}, state, { + isDirty: true, + query: action.payload + }); + + case Actions.QUERY_UPDATE_ERROR_MESSAGE: + return Object.assign({}, state, { errorMessage: action.message }); + + case Actions.QUERY_UPDATE_WARNING_MESSAGE: + return Object.assign({}, state, { warningMessage: action.message }); + + default: + return state; + } +}; diff --git a/Website/Packages/datax-query/src/modules/query/querySelectors.js b/Website/Packages/datax-query/src/modules/query/querySelectors.js new file mode 100644 index 000000000..54e07417a --- /dev/null +++ b/Website/Packages/datax-query/src/modules/query/querySelectors.js @@ -0,0 +1,30 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +import { createSelector } from 'reselect'; + +// Settings - query +export const getQuery = state => state.query; + +// Settings - Query +export const getQueryContent = createSelector( + getQuery, + query => query.query +); + +// Validation - Query +export const getQueryDirty = createSelector( + getQuery, + query => query.isDirty +); + +// Validation - Query +export const validateQueryTab = createSelector( + getQueryContent, + validateQuery +); + +function validateQuery(query) { + return query? (query || query.trim() === ''): true; +} diff --git a/Website/Packages/datax-query/src/styles/styles.css b/Website/Packages/datax-query/src/styles/styles.css new file mode 100644 index 000000000..a39415d71 --- /dev/null +++ b/Website/Packages/datax-query/src/styles/styles.css @@ -0,0 +1,27 @@ +/* + ********************************************************************* + Copyright (c) Microsoft Corporation. All rights reserved. + Licensed under the MIT License + ********************************************************************* +*/ + +/* Query Pane styles */ +.query-pane-button { + height: 27px !important; + padding-left: 5px !important; + padding-right: 5px !important; + min-width: 1px !important; + } + +.ms-IconExample:hover { + cursor: pointer !important; +} + +.query-pane-TextField{ + height: 27px !important; + width: 50px !important; +} + +.query-pane-TextField .ms-TextField-fieldGroup { + height: 27px !important; +} \ No newline at end of file diff --git a/Website/Packages/datax-query/webpack.config.js b/Website/Packages/datax-query/webpack.config.js new file mode 100644 index 000000000..f39db8bc2 --- /dev/null +++ b/Website/Packages/datax-query/webpack.config.js @@ -0,0 +1,130 @@ +// ********************************************************************* +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License +// ********************************************************************* +const webpack = require('webpack'); +const path = require('path'); +const pkg = require('./package.json'); +const MiniCssExtractPlugin = require('mini-css-extract-plugin'); +const MonacoWebpackPlugin = require('monaco-editor-webpack-plugin'); +const TerserPlugin = require('terser-webpack-plugin'); + +const libraryName = pkg.name; + +function configure(env, argv) { + const isDebug = argv.mode !== 'production'; + + var config = { + // https://webpack.js.org/configuration/entry-context/#entry + entry: { + index: './src/index.js' + }, + // https://webpack.js.org/configuration/output/ + output: { + path: path.resolve(__dirname, 'dist'), + filename: '[name].js', + library: libraryName, + libraryTarget: 'umd', + umdNamedDefine: true + }, + // https://webpack.js.org/configuration/resolve/ + resolve: { + extensions: ['.js', '.jsx'], + modules: [path.resolve(__dirname, 'src'), 'node_modules'], + alias: { 'react': path.resolve(__dirname, './node_modules/', 'react'), + 'react-dom': path.resolve('./node_modules/react-dom') + } + }, + // https://webpack.js.org/configuration/externals/ + externals: { + react: 'umd react', + 'react-dom': 'umd react-dom' + }, + // https://webpack.js.org/configuration/module/ + module: { + rules: [ + { + test: /\.(js|jsx)$/, + exclude: /(node_modules|bower_components)/, + use: { + // https://webpack.js.org/loaders/babel-loader/ + loader: 'babel-loader', + options: { + cacheDirectory: isDebug, + presets: [ + // https://babeljs.io/docs/en/babel-preset-env + '@babel/preset-env', + // https://babeljs.io/docs/en/babel-preset-react + '@babel/preset-react' + ], + plugins: [ + //https://babeljs.io/docs/en/babel-plugin-syntax-dynamic-import + '@babel/plugin-syntax-dynamic-import' + ] + } + } + }, + { + // https://github.com/webpack-contrib/mini-css-extract-plugin + test: /\.css$/, + use: [ + { + loader: MiniCssExtractPlugin.loader, + options: { + publicPath: '../' + } + }, + 'css-loader' + ] + }, + { + test: /\.svg(\?v=\d+\.\d+\.\d+)?$/, + loader: 'url-loader?name=img/[name].[ext]&limit=10000&mimetype=image/svg+xml' + }, + { + test: /\.eot(\?v=\d+\.\d+\.\d+)?$/, + loader: 'file-loader?name=fonts/[name].[ext]' + }, + { + test: /\.(woff|woff2)$/, + loader: 'url-loader?name=fonts/[name].[ext]&prefix=font&limit=5000' + }, + { + test: /\.ttf(\?v=\d+\.\d+\.\d+)?$/, + loader: 'url-loader?name=fonts/[name].[ext]&limit=10000&mimetype=application/octet-stream' + } + ] + }, + //https://webpack.js.org/configuration/plugins/ + plugins: [ + // https://github.com/Microsoft/monaco-editor-webpack-plugin + new MonacoWebpackPlugin({ + languages: ['sql'] + }), + // https://github.com/webpack-contrib/mini-css-extract-plugin + new MiniCssExtractPlugin({ + filename: 'css/[name].css' + }), + // https://webpack.js.org/plugins/limit-chunk-count-plugin/ + new webpack.optimize.LimitChunkCountPlugin({ + maxChunks: 1 + }) + ] + }; + + if (!isDebug) { + // https://github.com/webpack-contrib/terser-webpack-plugin + config.plugins.push( + new TerserPlugin({ + parallel: true, + terserOptions: { + mangle: true /* uglify */ + } + }) + ); + } + + return config; +} + +module.exports = configure; diff --git a/Website/Website/README.md b/Website/Website/README.md index 7c141764a..3746c8b17 100644 --- a/Website/Website/README.md +++ b/Website/Website/README.md @@ -62,7 +62,7 @@ faster performance for production consumption. npm run build ``` -### Deploy production version of website with zip-push otpion on Azure Web App +### Deploy production version of website with zip-push option on Azure Web App * Before deploying download and install AzureCLI, login first to get access for deployment: ``` @@ -85,3 +85,12 @@ set DATAXDEV_INTERACTIVE_QUERY_LOCAL_SERVICE= set DATAXDEV_LIVE_DATA_LOCAL_SERVICE= ``` + +### (Optional) How to target services hosted on AKS cluster +If you are hosting DataX services on an AKS cluster, you can configure the website to target the services hosted on an AKS cluster by defining the following secret in the kvServices keyvault. +Secret Name: +"kubernetesServices" where you need to use the prefix: ) from the step 4. +Secret Value: In the secret value: You only need to define the ones you are hosting on AKS cluster. +The should be added for each of the services where they can be listened to on the AKS cluster as specified in the example below: +``` +{"Flow.InteractiveQueryService":"http://:5000","Flow.SchemaInferenceService":"http://:5000","Flow.ManagementService":"http://:5000","Flow.LiveDataService":"http://:5000"} +``` diff --git a/Website/Website/auth.js b/Website/Website/auth.js index 43a4404f9..da4a74cae 100644 --- a/Website/Website/auth.js +++ b/Website/Website/auth.js @@ -186,12 +186,13 @@ function initialize(host) { }; /** - * call services on service fabric + * call services on service fabric or services hosted on Kubernetes * @param {http request coming from client} req * @param {query object to call Service Fabric service} query * Example: * { - * application: "DataX.Flow", + * application: "DataX.Flow",// This does not apply the kubernetes scenario. This applies only to the ServiceFabric scenario. + * the rest of the parameters stay the same for the Kubernetes scenario as well. * service: "Flow.ManagementService", * method: "GET", // or POST * headers: {"Content-type": "application/json"} // optional headers @@ -207,6 +208,8 @@ function initialize(host) { let url; if (env.localServices[query.service]) { url = `${env.localServices[query.service]}/api/${query.api}`; + } else if (env.kubernetesServices && env.kubernetesServices[query.service]) { + url = `${env.kubernetesServices[query.service]}/api/${query.api}`; } else { url = `${serviceClusterUrl}/api/${query.application}/${query.service}/${query.api}`; } @@ -345,6 +348,10 @@ exports.initialize = function(host) { res.type('application/json').send(functionEnabled(req.user._json.roles, host.conf.env.enableLocalOneBox)); }); + app.get('/api/isdatabrickssparktype', function(req, res) { + res.type('application/json').send(isDatabricksSparkType(process.env.DATAX_SPARK_TYPE)); + }); + app.all('/api/*', (req, res, next) => { let roles = req.user._json.roles; let errorResult = checkPermission(roles, webComposition.api, req); @@ -424,3 +431,7 @@ function functionEnabled(roles, enableLocalOneBox) { return supportedFunctionalities; } + +function isDatabricksSparkType(sparkType) { + return sparkType && sparkType.toLowerCase() == 'databricks' ? true : false; +} diff --git a/Website/Website/deploy/DataX.Utilities/DataX.Utility.CodeSign/DataX.Utility.CodeSign.csproj b/Website/Website/deploy/DataX.Utilities/DataX.Utility.CodeSign/DataX.Utility.CodeSign.csproj index 2a56ce979..c4ade7d7c 100644 --- a/Website/Website/deploy/DataX.Utilities/DataX.Utility.CodeSign/DataX.Utility.CodeSign.csproj +++ b/Website/Website/deploy/DataX.Utilities/DataX.Utility.CodeSign/DataX.Utility.CodeSign.csproj @@ -2,7 +2,7 @@ Library - netcoreapp2.1 + netcoreapp2.2 true diff --git a/Website/Website/package.json b/Website/Website/package.json index a379c0ddf..686e2c54c 100644 --- a/Website/Website/package.json +++ b/Website/Website/package.json @@ -63,11 +63,11 @@ "css-loader": "1.0.1", "file-loader": "2.0.0", "mini-css-extract-plugin": "0.4.4", - "datax-common": "1.2.2", - "datax-home": "1.2.2", - "datax-jobs": "1.2.2", - "datax-metrics": "1.2.2", - "datax-pipeline": "1.2.2", + "datax-common": "1.3.0", + "datax-home": "1.3.0", + "datax-jobs": "1.3.0", + "datax-metrics": "1.3.0", + "datax-pipeline": "1.3.0", "office-ui-fabric-react": "6.111.2", "promise-polyfill": "8.1.0", "prop-types": "15.6.2", diff --git a/Website/Website/securedSettings.js b/Website/Website/securedSettings.js index 5c02877e3..fd30ed1fd 100644 --- a/Website/Website/securedSettings.js +++ b/Website/Website/securedSettings.js @@ -51,6 +51,15 @@ module.exports = async function(host) { env.mongoDbUrl = await getSecretOrThrow('mongoDbUrl'); env.mongoSharedDbUrl = await getSecretOrThrow('mongoSharedDbUrl').catch(err => env.mongoDbUrl); }, + async function() { + let kubernetesServices = await getSecret('kubernetesServices'); + // Secret Name: +"kubernetesServices" where you need to use the prefix: ) needs to be specified in the keyvault on the azure portal that starts with kvServices + // Secret value is a JSON object looks like this: + // {"Flow.InteractiveQueryService":"http://:5000","Flow.SchemaInferenceService":"http://:5000","Flow.ManagementService":"http://:5000","Flow.LiveDataService":"http://:5000"} + if (kubernetesServices) { + env.kubernetesServices = JSON.parse(kubernetesServices); + } + }, async function() { let redisDataConnectionString = await getSecret('redisDataConnectionString'); if (redisDataConnectionString) diff --git a/Website/Website/web.composition.json b/Website/Website/web.composition.json index 0bf3924e2..81a38c8ea 100644 --- a/Website/Website/web.composition.json +++ b/Website/Website/web.composition.json @@ -114,6 +114,7 @@ "^DataX.Flow\/Flow.ManagementService\/job\/getall$", "^DataX.Flow\/Flow.ManagementService\/job\/getbynames$", "^DataX.Flow\/Flow.ManagementService\/flow\/save$", + "^DataX.Flow\/Flow.ManagementService\/flow\/deploy$", "^DataX.Flow\/Flow.ManagementService\/flow\/delete$", "^DataX.Flow\/Flow.ManagementService\/flow\/generateconfigs$", "^DataX.Flow\/Flow.ManagementService\/flow\/startjobs$", @@ -155,6 +156,7 @@ "enabledForWriter": [ "newFlowButtonEnabled", "saveFlowButtonEnabled", + "deployFlowButtonEnabled", "deleteFlowButtonEnabled", "getInputSchemaButtonEnabled", "outputSideToolBarEnabled", @@ -183,6 +185,9 @@ "deleteFunctionButtonEnabled", "addOutputSinkButtonEnabled", "deleteOutputSinkButtonEnabled", + "deleteFunctionButtonEnabled", + "addBatchButtonEnabled", + "deleteBatchButtonEnabled", "scaleNumExecutorsSliderEnabled", "scaleExecutorMemorySliderEnabled", "jobActionsEnabled"