codope commented on code in PR #8076:
URL: https://github.com/apache/hudi/pull/8076#discussion_r1196024827


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala:
##########
@@ -599,138 +582,250 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
     }
   }
 
-  test("Test bulk insert") {
+  test("Test bulk insert with insert into for single partitioned table") {
     withSQLConf("hoodie.sql.insert.mode" -> "non-strict") {
       withRecordType()(withTempDir { tmp =>
         Seq("cow", "mor").foreach {tableType =>
-          // Test bulk insert for single partition
-          val tableName = generateTableName
-          spark.sql(
-            s"""
-               |create table $tableName (
-               |  id int,
-               |  name string,
-               |  price double,
-               |  dt string
-               |) using hudi
-               | tblproperties (
-               |  type = '$tableType',
-               |  primaryKey = 'id'
-               | )
-               | partitioned by (dt)
-               | location '${tmp.getCanonicalPath}/$tableName'
-       """.stripMargin)
-          spark.sql("set hoodie.datasource.write.insert.drop.duplicates = 
false")
+          withTable(generateTableName) { tableName =>
+            spark.sql(
+              s"""
+                 |create table $tableName (
+                 |  id int,
+                 |  name string,
+                 |  price double,
+                 |  dt string
+                 |) using hudi
+                 | tblproperties (
+                 |  type = '$tableType',
+                 |  primaryKey = 'id'
+                 | )
+                 | partitioned by (dt)
+                 | location '${tmp.getCanonicalPath}/$tableName'
+         """.stripMargin)
+            spark.sql("set hoodie.datasource.write.insert.drop.duplicates = 
false")
 
-          // Enable the bulk insert
-          spark.sql("set hoodie.sql.bulk.insert.enable = true")
-          spark.sql(s"insert into $tableName values(1, 'a1', 10, 
'2021-07-18')")
+            // Enable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = true")
+            spark.sql(s"insert into $tableName values(1, 'a1', 10, 
'2021-07-18')")
 
-          assertResult(WriteOperationType.BULK_INSERT) {
-            getLastCommitMetadata(spark, 
s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            assertResult(WriteOperationType.BULK_INSERT) {
+              getLastCommitMetadata(spark, 
s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            }
+            checkAnswer(s"select id, name, price, dt from $tableName")(
+              Seq(1, "a1", 10.0, "2021-07-18")
+            )
+
+            // Disable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = false")
+            spark.sql(s"insert into $tableName values(2, 'a2', 10, 
'2021-07-18')")
+
+            assertResult(WriteOperationType.INSERT) {
+              getLastCommitMetadata(spark, 
s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            }
+            checkAnswer(s"select id, name, price, dt from $tableName order by 
id")(
+              Seq(1, "a1", 10.0, "2021-07-18"),
+              Seq(2, "a2", 10.0, "2021-07-18")
+            )
           }
-          checkAnswer(s"select id, name, price, dt from $tableName")(
-            Seq(1, "a1", 10.0, "2021-07-18")
-          )
+        }
+      })
+    }
+  }
 
-          // Disable the bulk insert
-          spark.sql("set hoodie.sql.bulk.insert.enable = false")
-          spark.sql(s"insert into $tableName values(2, 'a2', 10, 
'2021-07-18')")
+  test("Test bulk insert with insert into for multi partitioned table") {
+    withSQLConf("hoodie.sql.insert.mode" -> "non-strict") {
+      withRecordType()(withTempDir { tmp =>
+        Seq("cow", "mor").foreach { tableType =>
+          withTable(generateTableName) { tableMultiPartition =>
+            spark.sql(
+              s"""
+                 |create table $tableMultiPartition (
+                 |  id int,
+                 |  name string,
+                 |  price double,
+                 |  dt string,
+                 |  hh string
+                 |) using hudi
+                 | tblproperties (
+                 |  type = '$tableType',
+                 |  primaryKey = 'id'
+                 | )
+                 | partitioned by (dt, hh)
+                 | location '${tmp.getCanonicalPath}/$tableMultiPartition'
+         """.stripMargin)
 
-          assertResult(WriteOperationType.INSERT) {
-            getLastCommitMetadata(spark, 
s"${tmp.getCanonicalPath}/$tableName").getOperationType
+            // Enable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = true")
+            spark.sql(s"insert into $tableMultiPartition values(1, 'a1', 10, 
'2021-07-18', '12')")
+
+            checkAnswer(s"select id, name, price, dt, hh from 
$tableMultiPartition")(
+              Seq(1, "a1", 10.0, "2021-07-18", "12")
+            )
+            // Disable the bulk insert
+            spark.sql("set hoodie.sql.bulk.insert.enable = false")
+            spark.sql(s"insert into $tableMultiPartition " +
+              s"values(2, 'a2', 10, '2021-07-18','12')")
+
+            checkAnswer(s"select id, name, price, dt, hh from 
$tableMultiPartition order by id")(
+              Seq(1, "a1", 10.0, "2021-07-18", "12"),
+              Seq(2, "a2", 10.0, "2021-07-18", "12")
+            )
           }
-          checkAnswer(s"select id, name, price, dt from $tableName order by 
id")(
-            Seq(1, "a1", 10.0, "2021-07-18"),
-            Seq(2, "a2", 10.0, "2021-07-18")
-          )
+        }
+      })
+    }
+  }
 
-          // Test bulk insert for multi-level partition
-          val tableMultiPartition = generateTableName
-          spark.sql(
-            s"""
-               |create table $tableMultiPartition (
-               |  id int,
-               |  name string,
-               |  price double,
-               |  dt string,
-               |  hh string
-               |) using hudi
-               | tblproperties (
-               |  type = '$tableType',
-               |  primaryKey = 'id'
-               | )
-               | partitioned by (dt, hh)
-               | location '${tmp.getCanonicalPath}/$tableMultiPartition'
-       """.stripMargin)
+  test("Test bulk insert with insert into for non partitioned table") {

Review Comment:
   These tests are only testing for default values of 
`BULKINSERT_OVERWRITE_OPERATION_TYPE` right? Can we also test for the other 
possible value?



##########
hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala:
##########
@@ -106,8 +106,14 @@ private class HoodieV1WriteBuilder(writeOptions: 
CaseInsensitiveStringMap,
     override def toInsertableRelation: InsertableRelation = {
       new InsertableRelation {
         override def insert(data: DataFrame, overwrite: Boolean): Unit = {
+          val mode = if (overwriteTable || overwritePartition) {

Review Comment:
   Can you confirm if it's insert_overwrite_table then then table basePath will 
still be removed?



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertCommitActionExecutor.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.commit;
+
+import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.config.HoodieInternalConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.internal.DataSourceInternalWriterHelper;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SaveMode;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class DatasetBulkInsertCommitActionExecutor extends 
BaseDatasetBulkInsertCommitActionExecutor {
+
+  public DatasetBulkInsertCommitActionExecutor(HoodieWriteConfig config,
+                                               SparkRDDWriteClient writeClient,
+                                               String instantTime) {
+    super(config, writeClient, instantTime);
+  }
+
+  @Override
+  protected void preExecute() {
+    // no op
+  }
+
+  @Override
+  protected HoodieData<WriteStatus> doExecute(Dataset<Row> records, boolean 
arePartitionRecordsSorted) {
+    Map<String, String> opts = 
writeConfig.getProps().entrySet().stream().collect(Collectors.toMap(
+        e -> String.valueOf(e.getKey()),
+        e -> String.valueOf(e.getValue())));
+    Map<String, String> optsOverrides = Collections.singletonMap(
+        HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED, 
String.valueOf(arePartitionRecordsSorted));
+
+    String targetFormat;
+    Map<String, String> customOpts = new HashMap<>(1);
+    if (HoodieSparkUtils.isSpark2()) {
+      targetFormat = "org.apache.hudi.internal";
+    } else if (HoodieSparkUtils.isSpark3()) {
+      targetFormat = "org.apache.hudi.spark3.internal";
+      
customOpts.put(HoodieInternalConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key(), 
records.schema().json());
+    } else {
+      throw new HoodieException("Bulk insert using row writer is not supported 
with current Spark version."
+          + " To use row writer please switch to spark 2 or spark 3");
+    }
+
+    records.write().format(targetFormat)
+        .option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, 
instantTime)
+        .options(opts)
+        .options(customOpts)
+        .options(optsOverrides)
+        .mode(SaveMode.Append)
+        .save();
+    return null;

Review Comment:
   why return null here?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -850,9 +854,12 @@ object HoodieSparkSqlWriter {
     if (operation != WriteOperationType.DELETE) {
       if (mode == SaveMode.ErrorIfExists && tableExists) {
         throw new HoodieException(s"hoodie table at $tablePath already 
exists.")
-      } else if (mode == SaveMode.Overwrite && tableExists && operation != 
WriteOperationType.INSERT_OVERWRITE_TABLE) {
-        // When user set operation as INSERT_OVERWRITE_TABLE,
-        // overwrite will use INSERT_OVERWRITE_TABLE operator in 
doWriteOperation
+      } else if (mode == SaveMode.Overwrite && tableExists &&

Review Comment:
   Do you mean, for `Overwrite` mode, we should not delete the basePath. Just 
overwrite the existing data. If so, I agree with you. Probably something to 
tackle in another PR.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -772,66 +772,71 @@ object HoodieSparkSqlWriter {
     }
   }
 
-  def bulkInsertAsRow(sqlContext: SQLContext,
+  def bulkInsertAsRow(writeClient: SparkRDDWriteClient[_],
+                      parameters: Map[String, String],
                       hoodieConfig: HoodieConfig,
                       df: DataFrame,
+                      mode: SaveMode,
                       tblName: String,
                       basePath: Path,
-                      path: String,
                       instantTime: String,
                       writerSchema: Schema,
-                      isTablePartitioned: Boolean): (Boolean, 
common.util.Option[String]) = {
+                      tableConfig: HoodieTableConfig):
+  (Boolean, HOption[String], HOption[String], HOption[String], 
SparkRDDWriteClient[_], HoodieTableConfig) = {
     if (hoodieConfig.getBoolean(INSERT_DROP_DUPS)) {
       throw new HoodieException("Dropping duplicates with bulk_insert in row 
writer path is not supported yet")
     }
+    val sqlContext = 
writeClient.getEngineContext.asInstanceOf[HoodieSparkEngineContext].getSqlContext
+    val jsc = 
writeClient.getEngineContext.asInstanceOf[HoodieSparkEngineContext].getJavaSparkContext
 
     val writerSchemaStr = writerSchema.toString
 
-    val opts = hoodieConfig.getProps.toMap ++
+    // Make opts mutable since it could be modified by 
tryOverrideParquetWriteLegacyFormatProperty
+    val opts = mutable.Map() ++ hoodieConfig.getProps.toMap ++
       Map(HoodieWriteConfig.AVRO_SCHEMA_STRING.key -> writerSchemaStr)
 
-    val writeConfig = DataSourceUtils.createHoodieConfig(writerSchemaStr, 
path, tblName, mapAsJavaMap(opts))
-    val populateMetaFields = 
hoodieConfig.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS)
-
-    val bulkInsertPartitionerRows: BulkInsertPartitioner[Dataset[Row]] = if 
(populateMetaFields) {
-      val userDefinedBulkInsertPartitionerOpt = 
DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig)
-      if (userDefinedBulkInsertPartitionerOpt.isPresent) {
-        userDefinedBulkInsertPartitionerOpt.get
-      } else {
-        BulkInsertInternalPartitionerWithRowsFactory.get(writeConfig, 
isTablePartitioned)
-      }
-    } else {
-      // Sort modes are not yet supported when meta fields are disabled
-      new NonSortPartitionerWithRows()
+    // Auto set the value of "hoodie.parquet.writelegacyformat.enabled"
+    tryOverrideParquetWriteLegacyFormatProperty(opts, 
convertAvroSchemaToStructType(writerSchema))
+    val writeConfig = DataSourceUtils.createHoodieConfig(writerSchemaStr, 
basePath.toString, tblName, opts)
+    val executor = mode match {
+      case SaveMode.Append =>
+        new DatasetBulkInsertCommitActionExecutor(writeConfig, writeClient, 
instantTime)
+      case SaveMode.Overwrite =>
+        WriteOperationType.fromValue(hoodieConfig
+          
.getStringOrDefault(HoodieInternalConfig.BULKINSERT_OVERWRITE_OPERATION_TYPE)) 
match {
+          case WriteOperationType.INSERT_OVERWRITE =>
+            new DatasetBulkInsertOverwriteCommitActionExecutor(writeConfig, 
writeClient, instantTime)
+          case WriteOperationType.INSERT_OVERWRITE_TABLE =>
+            new 
DatasetBulkInsertOverwriteTableCommitActionExecutor(writeConfig, writeClient, 
instantTime);
+        }
+      case _ =>
+        throw new HoodieException(s"$mode with bulk_insert in row writer path 
is not supported yet");
     }
 
-    val shouldDropPartitionColumns = 
hoodieConfig.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS)
-    val hoodieDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(df, 
writeConfig, bulkInsertPartitionerRows, shouldDropPartitionColumns)
+    val writeResult = executor.execute(df, tableConfig.isTablePartitioned)
 
-    val optsOverrides = Map(
-      HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED ->
-        bulkInsertPartitionerRows.arePartitionRecordsSorted().toString
-    )
+    try {
+      val (writeSuccessful, compactionInstant, clusteringInstant) = mode match 
{
+        case SaveMode.Append =>
+          val syncHiveSuccess = metaSync(sqlContext.sparkSession, writeConfig, 
basePath, df.schema)
+          (syncHiveSuccess, HOption.empty().asInstanceOf[HOption[String]], 
HOption.empty().asInstanceOf[HOption[String]])
+        case SaveMode.Overwrite =>
+          try {
+            commitAndPerformPostOperations(sqlContext.sparkSession, df.schema, 
writeResult, parameters, writeClient, tableConfig, jsc,
+                TableInstantInfo(basePath, instantTime, 
executor.getCommitActionType, executor.getWriteOperationType), Option.empty)
 
-    val (targetFormat, customOpts) = if (HoodieSparkUtils.isSpark2) {
-      ("org.apache.hudi.internal", Map())
-    } else if (HoodieSparkUtils.isSpark3) {
-      ("org.apache.hudi.spark3.internal", Map(
-        HoodieInternalConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key -> 
hoodieDF.schema.json
-      ))
-    } else {
-      throw new HoodieException("Bulk insert using row writer is not supported 
with current Spark version."
-        + " To use row writer please switch to spark 2 or spark 3")
+          }
+      }
+      (writeSuccessful, common.util.Option.ofNullable(instantTime), 
compactionInstant, clusteringInstant, writeClient, tableConfig)

Review Comment:
   use `HOption.ofNullable`?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java:
##########
@@ -77,8 +78,11 @@ public HoodieData<WriteStatus> 
performClusteringWithRecordsAsRow(Dataset<Row> in
     // Since clustering will write to single file group using 
HoodieUnboundedCreateHandle, set max file size to a large value.
     newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, 
String.valueOf(Long.MAX_VALUE));
 
-    return HoodieDatasetBulkInsertHelper.bulkInsert(inputRecords, instantTime, 
getHoodieTable(), newConfig,
-        getRowPartitioner(strategyParams, schema), numOutputGroups, 
shouldPreserveHoodieMetadata);
+    BulkInsertPartitioner<Dataset<Row>> partitioner = 
getRowPartitioner(strategyParams, schema);
+    Dataset<Row> repartitionedRecords = 
partitioner.repartitionRecords(inputRecords, numOutputGroups);

Review Comment:
   Got it. Thanks for the explanation.



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/BaseDatasetBulkInsertCommitActionExecutor.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.commit;
+
+import org.apache.hudi.DataSourceUtils;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.HoodieDatasetBulkInsertHelper;
+import org.apache.hudi.client.HoodieWriteResult;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.CommitUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieException;
+import 
org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerWithRowsFactory;
+import org.apache.hudi.execution.bulkinsert.NonSortPartitionerWithRows;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public abstract class BaseDatasetBulkInsertCommitActionExecutor implements 
Serializable {
+
+  protected final transient HoodieWriteConfig writeConfig;
+  protected final transient SparkRDDWriteClient writeClient;
+  protected final String instantTime;
+  protected HoodieTable table;
+
+  public BaseDatasetBulkInsertCommitActionExecutor(HoodieWriteConfig config,
+                                                   SparkRDDWriteClient 
writeClient,
+                                                   String instantTime) {
+    this.writeConfig = config;
+    this.writeClient = writeClient;
+    this.instantTime = instantTime;
+  }
+
+  protected void preExecute() {
+    table.validateInsertSchema();
+    writeClient.startCommitWithTime(instantTime, getCommitActionType());
+    writeClient.preWrite(instantTime, getWriteOperationType(), 
table.getMetaClient());
+  }
+
+  protected abstract HoodieData<WriteStatus> doExecute(Dataset<Row> records, 
boolean arePartitionRecordsSorted);
+
+  protected void afterExecute(HoodieWriteMetadata<JavaRDD<WriteStatus>> 
result) {
+    writeClient.postWrite(result, instantTime, table);
+  }
+
+  private HoodieWriteMetadata<JavaRDD<WriteStatus>> 
buildHoodieWriteMetadata(HoodieData<WriteStatus> writeStatuses) {
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> hoodieWriteMetadata = new 
HoodieWriteMetadata<>();
+    if (writeStatuses != null) {
+      
hoodieWriteMetadata.setWriteStatuses(HoodieJavaRDD.getJavaRDD(writeStatuses));
+      
hoodieWriteMetadata.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(writeStatuses));
+    }
+    return hoodieWriteMetadata;
+  }
+
+  public final HoodieWriteResult execute(Dataset<Row> records, boolean 
isTablePartitioned) {
+    if (writeConfig.getBoolean(DataSourceWriteOptions.INSERT_DROP_DUPS())) {
+      throw new HoodieException("Dropping duplicates with bulk_insert in row 
writer path is not supported yet");
+    }
+
+    boolean populateMetaFields = 
writeConfig.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS);
+
+    BulkInsertPartitioner<Dataset<Row>> bulkInsertPartitionerRows = 
getPartitioner(populateMetaFields, isTablePartitioned);
+    boolean shouldDropPartitionColumns = 
writeConfig.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS());
+    Dataset<Row> hoodieDF = 
HoodieDatasetBulkInsertHelper.prepareForBulkInsert(records, writeConfig, 
bulkInsertPartitionerRows, shouldDropPartitionColumns);
+
+    table = writeClient.initTable(WriteOperationType.INSERT_OVERWRITE, 
Option.ofNullable(instantTime));

Review Comment:
   Will the operation type not depend on the specific implementation of this 
base class? I don't see `execute` being overridden in subclasses.



-- 
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