juarezr commented on code in PR #57514:
URL: https://github.com/apache/airflow/pull/57514#discussion_r2484935572


##########
providers/google/src/airflow/providers/google/cloud/transfers/mssql_to_gcs.py:
##########
@@ -67,7 +67,7 @@ class MSSQLToGCSOperator(BaseSQLToGCSOperator):
 
     ui_color = "#e0a98c"
 
-    type_map = {2: "BOOLEAN", 3: "INTEGER", 4: "TIMESTAMP", 5: "NUMERIC"}
+    type_map = {2: "BOOL", 3: "INTEGER", 4: "TIMESTAMP", 5: "NUMERIC"}

Review Comment:
   ## Testing the fix for bit_fields
   
   After your advice, I have tested exporting a simple query to the formats: 
PARQUET, CSV, and JSON.
   
   I also looked into the source code to understand what the map used in the 
boolean fields and found that the majority of cases use a map key `BOOL`. 
However, I also found there are some places that are mapping to `BOOLEAN` and 
could present the same issue as I've described. Please see for details below.
   
   ### Test Results
   
   To test the three formats, I've used a simple DAG with two tasks: One task 
to export the data from MSSQL to GCS and another task to load the data into 
BigQuery.
   
   I've run this DAG once for each format. In each run, the data was exported 
correctly and the schema was generated correctly with the correct data type for 
the boolean fields in the parquet format.
   
   Here is the diagram of the data pipeline used to export the data from MSSQL 
to GCS and then to BigQuery:
   
   ```plain
   ┌─────────┐                                      ┌─────────┐                 
                     ┌──────────┐
   │         ││    ┌─────────────────────────┐      │         ││    
┌────────────────────────────┐   │          ││
   │  MSSQL  ││    │                         │      │   GCS   ││    │           
                 │   │ BigQuery ││
   │      ┣──┼─────┼─► MSSQLToGCSOperator ┣─┼──────┼──►   ┣─┼─────┼─► 
GCSToBigQueryOperator ┣─┼───┼──►      ││
   │         ││    │                         │      │         ││    │           
                 │   │          ││
   └─────────┘│    └─────────────────────────┘      └─────────┘│    
└────────────────────────────┘   └──────────┘│
    ──────────┘                                      ──────────┘                
                      ───────────┘
   ```
   
   ### Test Scenario
   
   #### Query
   
   ```sql
   SELECT id, name, isactive, clientid FROM dbo.DriverGroup WHERE id IN (9, 47, 
83)
   ```
   
   #### CSV
   
   This is what was exported to CSV:
   
   ```csv
   id,name,isactive,clientid
   9,OPERACIONAL,True,6947
   47,CLUSTER 13,True,7348
   83,Aero,False,25
   ```
   
   All tasks of this run executed successfully with no need to specify the 
schema for export/import.
   
   #### JSON
   
   This is what was exported to JSON:
   
   ```json
   {"clientid": 6947, "id": 9, "isactive": true, "name": "OPERACIONAL"}
   {"clientid": 7348, "id": 47, "isactive": true, "name": "CLUSTER 13"}
   {"clientid": 25, "id": 83, "isactive": false, "name": "Aero"}
   ```
   
   This run failed while importing the data in BigQuery due to the export are 
not preserving the field order as is stated in the SQL query, which is the same 
field order of the destination table in BigQuery.
   
   But I believe this is unrelated to the issue we've been discussing here. 
Also, this can be corrected with a schema mapping in the 
`GCSToBigQueryOperator` operator.
   
   #### Parquet
   
   Here is a copy/paste representation of the records exported to Parquet:
   
   ```plain
   id name isactive clientid
   9 OPERACIONAL true 6947
   47 CLUSTER 13 true 7348
   83 Aero false 25
   ```
   
   And this is the schema of the exported parquet file:
   
   ```csv
   Column name Data type Nullable Metadata
   id Int64 true {}
   name String true {}
   isactive Bool true {}
   clientid Int64 true {}
   ```
   
   ## Source Code Investigation
   
   ### Operators with BOOL mapping
   
   Looking in the source code, I found a couple of Operators that have the same 
target mapping defined to `BOOL` as the proposed fix. These Operators are:
   
   - PostgresToGCSOperator
   - TrinoToGCSOperator
   - CassandraToGCSOperator
   - PrestoToGCSOperator
   
   I've concluded that the fix is in the right direction, as these Operators 
are using the same target mapping defined to `BOOL` as the proposed fix. But I 
can be wrong in this conclusion.
   
   #### PostgresToGCSOperator
   
   ```python
   # 
providers/google/src/airflow/providers/google/cloud/transfers/postgres_to_gcs.py:112
   
   class PostgresToGCSOperator(BaseSQLToGCSOperator):
   
       type_map = {
           1114: "DATETIME",
           1184: "TIMESTAMP",
           1082: "DATE",
           1083: "TIME",
           1005: "INTEGER",
           1007: "INTEGER",
           1016: "INTEGER",
           20: "INTEGER",
           21: "INTEGER",
           23: "INTEGER",
           16: "BOOL", # <-------- MAPS boolean/16 to BOOL ---<
           700: "FLOAT",
           701: "FLOAT",
           1700: "FLOAT",
       }
   ```
   
   #### TrinoToGCSOperator
   
   ```python
   # 
providers/google/src/airflow/providers/google/cloud/transfers/trino_to_gcs.py:159
   
   class TrinoToGCSOperator(BaseSQLToGCSOperator):
   
       type_map = {
           "BOOLEAN": "BOOL",# <-------- MAPS BOOLEAN to BOOL ---<
           "TINYINT": "INT64",
           "SMALLINT": "INT64",
           "INTEGER": "INT64",
           "BIGINT": "INT64",
           "REAL": "FLOAT64",
           "DOUBLE": "FLOAT64",
           "DECIMAL": "NUMERIC",
           "VARCHAR": "STRING",
           "CHAR": "STRING",
           "VARBINARY": "BYTES",
           "JSON": "STRING",
           "DATE": "DATE",
           "TIME": "TIME",
           # BigQuery don't time with timezone native.
           "TIME WITH TIME ZONE": "STRING",
           "TIMESTAMP": "TIMESTAMP",
           # BigQuery supports a narrow range of time zones during import.
           # You should use TIMESTAMP function, if you want have TIMESTAMP type
           "TIMESTAMP WITH TIME ZONE": "STRING",
           "IPADDRESS": "STRING",
           "UUID": "STRING",
       }
   ```
   
   #### CassandraToGCSOperator
   
   ```python
   # 
providers/google/src/airflow/providers/google/cloud/transfers/cassandra_to_gcs.py:128
   
   class CassandraToGCSOperator(BaseOperator):
   
       CQL_TYPE_MAP = {
           "BytesType": "STRING",
           "DecimalType": "FLOAT",
           "UUIDType": "STRING",
           "BooleanType": "BOOL", # <-------- MAPS BOOLEAN to BOOL ---<
           "ByteType": "INTEGER",
           "AsciiType": "STRING",
           "FloatType": "FLOAT",
           "DoubleType": "FLOAT",
           "LongType": "INTEGER",
           "Int32Type": "INTEGER",
           "IntegerType": "INTEGER",
           "InetAddressType": "STRING",
           "CounterColumnType": "INTEGER",
           "DateType": "TIMESTAMP",
           "SimpleDateType": "DATE",
           "TimestampType": "TIMESTAMP",
           "TimeUUIDType": "STRING",
           "ShortType": "INTEGER",
           "TimeType": "TIME",
           "DurationType": "INTEGER",
           "UTF8Type": "STRING",
           "VarcharType": "STRING",
       }
   ```
   
   #### PrestoToGCSOperator
   
   ```python
   # 
providers/google/src/airflow/providers/google/cloud/transfers/presto_to_gcs.py:156
   
   class PrestoToGCSOperator(BaseSQLToGCSOperator):
   
       type_map = {
           "BOOLEAN": "BOOL",# <-------- MAPS BOOLEAN to BOOL ---<
           "TINYINT": "INT64",
           "SMALLINT": "INT64",
           "INTEGER": "INT64",
           "BIGINT": "INT64",
           "REAL": "FLOAT64",
           "DOUBLE": "FLOAT64",
           "DECIMAL": "NUMERIC",
           "VARCHAR": "STRING",
           "CHAR": "STRING",
           "VARBINARY": "BYTES",
           "JSON": "STRING",
           "DATE": "DATE",
           "TIME": "TIME",
           # BigQuery don't time with timezone native.
           "TIME WITH TIME ZONE": "STRING",
           "TIMESTAMP": "TIMESTAMP",
           # BigQuery supports a narrow range of time zones during import.
           # You should use TIMESTAMP function, if you want have TIMESTAMP type
           "TIMESTAMP WITH TIME ZONE": "STRING",
           "IPADDRESS": "STRING",
           "UUID": "STRING",
       }
   ```
   
   ### Operators with BOOLEAN mapping
   
   I've also found a couple of operators with the target mapping defined as 
`BOOLEAN`, unlike this proposed fix.
   
   I believe that there is a potential issue with these Operators. Its possible 
that they may have the same issue as the code before this fix because the type 
names will mismatch.
   
   However, I'm not sure if this is a real issue or not. Also, I don't have a 
readily available environment to test this hypothesis.
   
   #### OracleToGCSOperator
   
   ```python
   # 
providers/google/src/airflow/providers/google/cloud/transfers/oracle_to_gcs.py:57
   
   class OracleToGCSOperator(BaseSQLToGCSOperator):
   
       type_map = {
           oracledb.DB_TYPE_BINARY_DOUBLE: "DECIMAL",
           oracledb.DB_TYPE_BINARY_FLOAT: "DECIMAL",
           oracledb.DB_TYPE_BINARY_INTEGER: "INTEGER",
           oracledb.DB_TYPE_BOOLEAN: "BOOLEAN", # <-------- MAPS BOOLEAN to 
BOOLEAN ---<
           oracledb.DB_TYPE_DATE: "TIMESTAMP",
           oracledb.DB_TYPE_NUMBER: "NUMERIC",
           oracledb.DB_TYPE_TIMESTAMP: "TIMESTAMP",
           oracledb.DB_TYPE_TIMESTAMP_LTZ: "TIMESTAMP",
           oracledb.DB_TYPE_TIMESTAMP_TZ: "TIMESTAMP",
       }
   ```
   
   #### MySQLToGCSOperator
   
   Notice that MySQLToGCSOperator maps BIT to an INTEGER data type instead of 
BOOLEAN:
   
   ```python
   # 
providers/google/src/airflow/providers/google/cloud/transfers/mysql_to_gcs.py:63
   
   class MySQLToGCSOperator(BaseSQLToGCSOperator):
   
       type_map = {
           FIELD_TYPE.BIT: "INTEGER", # <-------- MAPS BIT to INTEGER ---<
           FIELD_TYPE.DATETIME: "TIMESTAMP",
           FIELD_TYPE.DATE: "TIMESTAMP",
           FIELD_TYPE.DECIMAL: "FLOAT",
           FIELD_TYPE.NEWDECIMAL: "FLOAT",
           FIELD_TYPE.DOUBLE: "FLOAT",
           FIELD_TYPE.FLOAT: "FLOAT",
           FIELD_TYPE.INT24: "INTEGER",
           FIELD_TYPE.LONG: "INTEGER",
           FIELD_TYPE.LONGLONG: "INTEGER",
           FIELD_TYPE.SHORT: "INTEGER",
           FIELD_TYPE.TIME: "TIME",
           FIELD_TYPE.TIMESTAMP: "TIMESTAMP",
           FIELD_TYPE.TINY: "INTEGER",
           FIELD_TYPE.YEAR: "INTEGER",
       }
   ```
   



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to