You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "jduo (via GitHub)" <gi...@apache.org> on 2023/11/15 22:04:27 UTC

Re: [PR] test(csharp): add more tests for verifying GetObjects (Depths and Patterns) [arrow-adbc]

jduo commented on code in PR #1299:
URL: https://github.com/apache/arrow-adbc/pull/1299#discussion_r1394876391


##########
csharp/test/Drivers/Snowflake/DriverTests.cs:
##########
@@ -112,10 +112,266 @@ public void CanGetInfo()
         }
 
         /// <summary>
-        /// Validates if the driver can call GetObjects.
+        /// Validates if the driver can call GetObjects with GetObjectsDepth as Catalogs.
         /// </summary>
         [SkippableFact, Order(3)]
-        public void CanGetObjects()
+        public void CanGetObjectsCatalogs()
+        {
+            Dictionary<string, string> parameters = new Dictionary<string, string>();
+
+            SnowflakeTestConfiguration testConfiguration = Utils.LoadTestConfiguration<SnowflakeTestConfiguration>(SnowflakeTestingUtils.SNOWFLAKE_TEST_CONFIG_VARIABLE);
+
+            AdbcDriver driver = SnowflakeTestingUtils.GetSnowflakeAdbcDriver(testConfiguration, out parameters);
+
+            string databaseName = testConfiguration.Metadata.Catalog;
+            string schemaName = testConfiguration.Metadata.Schema;
+
+            parameters[SnowflakeParameters.DATABASE] = databaseName;
+            parameters[SnowflakeParameters.SCHEMA] = schemaName;
+
+            AdbcDatabase adbcDatabase = driver.Open(parameters);
+            AdbcConnection adbcConnection = adbcDatabase.Connect(new Dictionary<string, string>());

Review Comment:
   It would be worth checking if some of these objects need to be wrapped in using() blocks



##########
csharp/test/Drivers/Snowflake/DriverTests.cs:
##########
@@ -112,10 +112,266 @@ public void CanGetInfo()
         }
 
         /// <summary>
-        /// Validates if the driver can call GetObjects.
+        /// Validates if the driver can call GetObjects with GetObjectsDepth as Catalogs.
         /// </summary>
         [SkippableFact, Order(3)]
-        public void CanGetObjects()
+        public void CanGetObjectsCatalogs()
+        {
+            Dictionary<string, string> parameters = new Dictionary<string, string>();
+
+            SnowflakeTestConfiguration testConfiguration = Utils.LoadTestConfiguration<SnowflakeTestConfiguration>(SnowflakeTestingUtils.SNOWFLAKE_TEST_CONFIG_VARIABLE);
+
+            AdbcDriver driver = SnowflakeTestingUtils.GetSnowflakeAdbcDriver(testConfiguration, out parameters);
+
+            string databaseName = testConfiguration.Metadata.Catalog;
+            string schemaName = testConfiguration.Metadata.Schema;
+
+            parameters[SnowflakeParameters.DATABASE] = databaseName;
+            parameters[SnowflakeParameters.SCHEMA] = schemaName;
+
+            AdbcDatabase adbcDatabase = driver.Open(parameters);
+            AdbcConnection adbcConnection = adbcDatabase.Connect(new Dictionary<string, string>());
+
+            IArrowArrayStream stream = adbcConnection.GetObjects(
+                    depth: AdbcConnection.GetObjectsDepth.Catalogs,
+                    catalogPattern: databaseName,
+                    dbSchemaPattern: null,
+                    tableNamePattern: null,
+                    tableTypes: new List<string> { "BASE TABLE", "VIEW" },
+                    columnNamePattern: null);
+
+            RecordBatch recordBatch = stream.ReadNextRecordBatchAsync().Result;
+
+            List<AdbcCatalog> catalogs = GetObjectsParser.ParseCatalog(recordBatch, databaseName, null);
+
+            AdbcCatalog catalog = catalogs.FirstOrDefault();
+
+            Assert.True(catalog != null, "catalog should not be null");
+            Assert.Equal(databaseName, catalog.Name);
+        }
+
+        /// <summary>
+        /// Validates if the driver can call GetObjects with GetObjectsDepth as Catalogs and CatalogName passed as a pattern.
+        /// </summary>
+        [SkippableFact, Order(3)]
+        public void CanGetObjectsCatalogsWithPattern()
+        {
+            Dictionary<string, string> parameters = new Dictionary<string, string>();
+
+            SnowflakeTestConfiguration testConfiguration = Utils.LoadTestConfiguration<SnowflakeTestConfiguration>(SnowflakeTestingUtils.SNOWFLAKE_TEST_CONFIG_VARIABLE);
+
+            AdbcDriver driver = SnowflakeTestingUtils.GetSnowflakeAdbcDriver(testConfiguration, out parameters);
+
+            string databaseName = testConfiguration.Metadata.Catalog;
+            string schemaName = testConfiguration.Metadata.Schema;
+            string partialDatabaseName = $"{GetPartialNameForPatternMatch(databaseName)}%";

Review Comment:
   We should test a variety of wildcards and also test if case-sensitivity (or insensitivity) is handled correctly.



##########
csharp/test/Drivers/Snowflake/DriverTests.cs:
##########
@@ -112,10 +112,266 @@ public void CanGetInfo()
         }
 
         /// <summary>
-        /// Validates if the driver can call GetObjects.
+        /// Validates if the driver can call GetObjects with GetObjectsDepth as Catalogs.
         /// </summary>
         [SkippableFact, Order(3)]
-        public void CanGetObjects()
+        public void CanGetObjectsCatalogs()
+        {
+            Dictionary<string, string> parameters = new Dictionary<string, string>();
+
+            SnowflakeTestConfiguration testConfiguration = Utils.LoadTestConfiguration<SnowflakeTestConfiguration>(SnowflakeTestingUtils.SNOWFLAKE_TEST_CONFIG_VARIABLE);
+
+            AdbcDriver driver = SnowflakeTestingUtils.GetSnowflakeAdbcDriver(testConfiguration, out parameters);
+
+            string databaseName = testConfiguration.Metadata.Catalog;
+            string schemaName = testConfiguration.Metadata.Schema;
+
+            parameters[SnowflakeParameters.DATABASE] = databaseName;
+            parameters[SnowflakeParameters.SCHEMA] = schemaName;
+
+            AdbcDatabase adbcDatabase = driver.Open(parameters);
+            AdbcConnection adbcConnection = adbcDatabase.Connect(new Dictionary<string, string>());
+
+            IArrowArrayStream stream = adbcConnection.GetObjects(
+                    depth: AdbcConnection.GetObjectsDepth.Catalogs,
+                    catalogPattern: databaseName,
+                    dbSchemaPattern: null,
+                    tableNamePattern: null,
+                    tableTypes: new List<string> { "BASE TABLE", "VIEW" },
+                    columnNamePattern: null);
+
+            RecordBatch recordBatch = stream.ReadNextRecordBatchAsync().Result;
+
+            List<AdbcCatalog> catalogs = GetObjectsParser.ParseCatalog(recordBatch, databaseName, null);
+
+            AdbcCatalog catalog = catalogs.FirstOrDefault();
+
+            Assert.True(catalog != null, "catalog should not be null");
+            Assert.Equal(databaseName, catalog.Name);
+        }
+
+        /// <summary>
+        /// Validates if the driver can call GetObjects with GetObjectsDepth as Catalogs and CatalogName passed as a pattern.
+        /// </summary>
+        [SkippableFact, Order(3)]
+        public void CanGetObjectsCatalogsWithPattern()
+        {
+            Dictionary<string, string> parameters = new Dictionary<string, string>();
+
+            SnowflakeTestConfiguration testConfiguration = Utils.LoadTestConfiguration<SnowflakeTestConfiguration>(SnowflakeTestingUtils.SNOWFLAKE_TEST_CONFIG_VARIABLE);
+
+            AdbcDriver driver = SnowflakeTestingUtils.GetSnowflakeAdbcDriver(testConfiguration, out parameters);
+
+            string databaseName = testConfiguration.Metadata.Catalog;
+            string schemaName = testConfiguration.Metadata.Schema;
+            string partialDatabaseName = $"{GetPartialNameForPatternMatch(databaseName)}%";
+
+            parameters[SnowflakeParameters.DATABASE] = databaseName;
+            parameters[SnowflakeParameters.SCHEMA] = schemaName;
+
+            AdbcDatabase adbcDatabase = driver.Open(parameters);
+            AdbcConnection adbcConnection = adbcDatabase.Connect(new Dictionary<string, string>());
+
+            IArrowArrayStream stream = adbcConnection.GetObjects(
+                    depth: AdbcConnection.GetObjectsDepth.Catalogs,
+                    catalogPattern: partialDatabaseName,
+                    dbSchemaPattern: null,
+                    tableNamePattern: null,
+                    tableTypes: new List<string> { "BASE TABLE", "VIEW" },
+                    columnNamePattern: null);
+
+            RecordBatch recordBatch = stream.ReadNextRecordBatchAsync().Result;
+
+            List<AdbcCatalog> catalogs = GetObjectsParser.ParseCatalog(recordBatch, databaseName, null);
+
+            AdbcCatalog catalog = catalogs.FirstOrDefault();
+
+            Assert.True(catalog != null, "catalog should not be null");
+        }
+
+        /// <summary>
+        /// Validates if the driver can call GetObjects with GetObjectsDepth as DbSchemas.
+        /// </summary>
+        [SkippableFact, Order(3)]
+        public void CanGetObjectsDbSchemas()
+        {
+            Dictionary<string, string> parameters = new Dictionary<string, string>();
+
+            SnowflakeTestConfiguration testConfiguration = Utils.LoadTestConfiguration<SnowflakeTestConfiguration>(SnowflakeTestingUtils.SNOWFLAKE_TEST_CONFIG_VARIABLE);
+
+            AdbcDriver driver = SnowflakeTestingUtils.GetSnowflakeAdbcDriver(testConfiguration, out parameters);
+
+            // need to add the database
+            string databaseName = testConfiguration.Metadata.Catalog;
+            string schemaName = testConfiguration.Metadata.Schema;
+
+            parameters[SnowflakeParameters.DATABASE] = databaseName;
+            parameters[SnowflakeParameters.SCHEMA] = schemaName;
+
+            AdbcDatabase adbcDatabase = driver.Open(parameters);
+            AdbcConnection adbcConnection = adbcDatabase.Connect(new Dictionary<string, string>());
+
+            IArrowArrayStream stream = adbcConnection.GetObjects(
+                    depth: AdbcConnection.GetObjectsDepth.DbSchemas,
+                    catalogPattern: databaseName,
+                    dbSchemaPattern: schemaName,
+                    tableNamePattern: null,
+                    tableTypes: new List<string> { "BASE TABLE", "VIEW" },
+                    columnNamePattern: null);
+
+            RecordBatch recordBatch = stream.ReadNextRecordBatchAsync().Result;
+
+            List<AdbcCatalog> catalogs = GetObjectsParser.ParseCatalog(recordBatch, databaseName, schemaName);
+
+            List<AdbcDbSchema> dbSchemas = catalogs
+                .Select(s => s.DbSchemas)
+                .FirstOrDefault();
+            AdbcDbSchema dbSchema = dbSchemas.FirstOrDefault();
+
+            Assert.True(dbSchema != null, "dbSchema should not be null");
+            Assert.Equal(schemaName, dbSchema.Name);
+        }
+
+        /// <summary>
+        /// Validates if the driver can call GetObjects with GetObjectsDepth as DbSchemas with DbSchemaName as a pattern.
+        /// </summary>
+        [SkippableFact, Order(3)]
+        public void CanGetObjectsDbSchemasWithPattern()
+        {
+            Dictionary<string, string> parameters = new Dictionary<string, string>();
+
+            SnowflakeTestConfiguration testConfiguration = Utils.LoadTestConfiguration<SnowflakeTestConfiguration>(SnowflakeTestingUtils.SNOWFLAKE_TEST_CONFIG_VARIABLE);
+
+            AdbcDriver driver = SnowflakeTestingUtils.GetSnowflakeAdbcDriver(testConfiguration, out parameters);
+
+            // need to add the database
+            string databaseName = testConfiguration.Metadata.Catalog;
+            string schemaName = testConfiguration.Metadata.Schema;
+            string partialSchemaName = $"{GetPartialNameForPatternMatch(schemaName)}%";
+
+            parameters[SnowflakeParameters.DATABASE] = databaseName;
+            parameters[SnowflakeParameters.SCHEMA] = schemaName;
+
+            AdbcDatabase adbcDatabase = driver.Open(parameters);
+            AdbcConnection adbcConnection = adbcDatabase.Connect(new Dictionary<string, string>());
+
+            IArrowArrayStream stream = adbcConnection.GetObjects(
+                    depth: AdbcConnection.GetObjectsDepth.DbSchemas,
+                    catalogPattern: databaseName,
+                    dbSchemaPattern: partialSchemaName,
+                    tableNamePattern: null,
+                    tableTypes: new List<string> { "BASE TABLE", "VIEW" },
+                    columnNamePattern: null);
+
+            RecordBatch recordBatch = stream.ReadNextRecordBatchAsync().Result;
+
+            List<AdbcCatalog> catalogs = GetObjectsParser.ParseCatalog(recordBatch, databaseName, schemaName);
+
+            List<AdbcDbSchema> dbSchemas = catalogs
+                .Select(s => s.DbSchemas)
+                .FirstOrDefault();
+            AdbcDbSchema dbSchema = dbSchemas.FirstOrDefault();
+
+            Assert.True(dbSchema != null, "dbSchema should not be null");
+        }
+
+        /// <summary>
+        /// Validates if the driver can call GetObjects with GetObjectsDepth as Tables.
+        /// </summary>
+        [SkippableFact, Order(3)]
+        public void CanGetObjectsTables()
+        {
+            Dictionary<string, string> parameters = new Dictionary<string, string>();
+
+            SnowflakeTestConfiguration testConfiguration = Utils.LoadTestConfiguration<SnowflakeTestConfiguration>(SnowflakeTestingUtils.SNOWFLAKE_TEST_CONFIG_VARIABLE);
+
+            AdbcDriver driver = SnowflakeTestingUtils.GetSnowflakeAdbcDriver(testConfiguration, out parameters);
+
+            // need to add the database
+            string databaseName = testConfiguration.Metadata.Catalog;
+            string schemaName = testConfiguration.Metadata.Schema;
+            string tableName = testConfiguration.Metadata.Table;
+
+            parameters[SnowflakeParameters.DATABASE] = databaseName;
+            parameters[SnowflakeParameters.SCHEMA] = schemaName;
+
+            AdbcDatabase adbcDatabase = driver.Open(parameters);
+            AdbcConnection adbcConnection = adbcDatabase.Connect(new Dictionary<string, string>());
+
+            IArrowArrayStream stream = adbcConnection.GetObjects(
+                    depth: AdbcConnection.GetObjectsDepth.All,
+                    catalogPattern: databaseName,
+                    dbSchemaPattern: schemaName,
+                    tableNamePattern: tableName,
+                    tableTypes: new List<string> { "BASE TABLE", "VIEW" },
+                    columnNamePattern: null);
+
+            RecordBatch recordBatch = stream.ReadNextRecordBatchAsync().Result;
+
+            List<AdbcCatalog> catalogs = GetObjectsParser.ParseCatalog(recordBatch, databaseName, schemaName);
+
+            List<AdbcTable> tables = catalogs
+                .Select(s => s.DbSchemas)
+                .FirstOrDefault()
+                .Select(t => t.Tables)
+                .FirstOrDefault();
+            AdbcTable table = tables.FirstOrDefault();
+
+            Assert.True(table != null, "table should not be null");
+            Assert.Equal(tableName, table.Name);
+        }
+
+        /// <summary>
+        /// Validates if the driver can call GetObjects with GetObjectsDepth as Tables with TableName as a pattern.
+        /// </summary>
+        [SkippableFact, Order(3)]
+        public void CanGetObjectsTablesWithPattern()
+        {
+            Dictionary<string, string> parameters = new Dictionary<string, string>();
+
+            SnowflakeTestConfiguration testConfiguration = Utils.LoadTestConfiguration<SnowflakeTestConfiguration>(SnowflakeTestingUtils.SNOWFLAKE_TEST_CONFIG_VARIABLE);
+
+            AdbcDriver driver = SnowflakeTestingUtils.GetSnowflakeAdbcDriver(testConfiguration, out parameters);
+
+            // need to add the database
+            string databaseName = testConfiguration.Metadata.Catalog;
+            string schemaName = testConfiguration.Metadata.Schema;
+            string tableName = testConfiguration.Metadata.Table;
+            string partialTableName = $"{GetPartialNameForPatternMatch(tableName)}%";
+
+            parameters[SnowflakeParameters.DATABASE] = databaseName;
+            parameters[SnowflakeParameters.SCHEMA] = schemaName;
+
+            AdbcDatabase adbcDatabase = driver.Open(parameters);
+            AdbcConnection adbcConnection = adbcDatabase.Connect(new Dictionary<string, string>());
+
+            IArrowArrayStream stream = adbcConnection.GetObjects(
+                    depth: AdbcConnection.GetObjectsDepth.All,
+                    catalogPattern: databaseName,
+                    dbSchemaPattern: schemaName,
+                    tableNamePattern: partialTableName,
+                    tableTypes: new List<string> { "BASE TABLE", "VIEW" },
+                    columnNamePattern: null);
+
+            RecordBatch recordBatch = stream.ReadNextRecordBatchAsync().Result;
+
+            List<AdbcCatalog> catalogs = GetObjectsParser.ParseCatalog(recordBatch, databaseName, schemaName);
+
+            List<AdbcTable> tables = catalogs
+                .Select(s => s.DbSchemas)
+                .FirstOrDefault()
+                .Select(t => t.Tables)
+                .FirstOrDefault();
+            AdbcTable table = tables.FirstOrDefault();
+
+            Assert.True(table != null, "table should not be null");

Review Comment:
   Let's validate that the table name matches up to the pattern (before any wildcards).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org