Skip to content

Conversation

@hililiwei
Copy link
Contributor

@hililiwei hililiwei commented Nov 14, 2021

#3499

  1. new enum named SparkCatalogType with three test spark catalog 'testhive''testhadoop''spark_catalog''
public enum SparkCatalogType {
  TEST_HIVE("testhive", SparkCatalog.class.getName(), ImmutableMap.of(
      "type", "hive",
      "default-namespace", "default"
  )),
  TEST_HADOOP("testhadoop", SparkCatalog.class.getName(), ImmutableMap.of(
      "type", "hadoop"
  )),
  SPARK_CATALOG("spark_catalog", SparkSessionCatalog.class.getName(), ImmutableMap.of(
      "type", "hive",
      "default-namespace", "default",
      "parquet-enabled", "true",
      "cache-enabled", "false" // Spark will delete tables using v1, leaving the cache out of sync
  ));
  1. new base abstract SparkSpecifyCatalogTestBase with three constructors
  public SparkSpecifyCatalogTestBase() {
    this(SparkCatalogType.TEST_HADOOP, null);
  }

  public SparkSpecifyCatalogTestBase(SparkCatalogType sparkCatalogType) {
    this(sparkCatalogType, null);
  }

  public SparkSpecifyCatalogTestBase(SparkCatalogType sparkCatalogType, Map<String, String> config) {
 .........
 }

Importantly, I'm not sure which test cases can be migrated to use it.

thx.

@github-actions github-actions bot added the spark label Nov 14, 2021
@hililiwei hililiwei force-pushed the ICEBERG-3499 branch 3 times, most recently from d6f996c to adacd56 Compare November 14, 2021 15:50
Copy link
Contributor

@kbendick kbendick left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for working on this @hililiwei! I just have one piece of input or opinion that I'd like to discuss (and hear other's opinions on if they care to chime in). Let a comment in the code on where that change might be made.

In some of the tests that extend SparkCatalogTestBase, the catalogs are overridden by a new @Parameters section with only one or more catalogs. Often times, currently, this is done to allow for overriding the catalog config. But everything has to be redeclared.

Here's an example where we overrode the @Parameters to run a different set of catalogs (though still used all 3). The main reason for the override here was to allow for changing the catalog configuration with the additional stuff:

@Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
public static Object[][] parameters() {
return new Object[][] {
{ "testhive", SparkCatalog.class.getName(),
ImmutableMap.of(
"type", "hive",
"default-namespace", "default",
hadoopPrefixedConfigToOverride, configOverrideValue
) },
{ "testhadoop", SparkCatalog.class.getName(),
ImmutableMap.of(
"type", "hadoop",
hadoopPrefixedConfigToOverride, configOverrideValue
) },
{ "spark_catalog", SparkSessionCatalog.class.getName(),
ImmutableMap.of(
"type", "hive",
"default-namespace", "default",
hadoopPrefixedConfigToOverride, configOverrideValue
) }
};
}
public TestSparkCatalogHadoopOverrides(String catalogName,
String implementation,
Map<String, String> config) {
super(catalogName, implementation, config);
}

If we could make it so that the config in the enum can be overridden without redeclaring everything (or whatever it winds up being if enums don't support that), that would be a big win in my view / really useful!

For me, it would be great if the passed in config could override the base configuration key by key, so things like type=hive don't need to be set again, that would be great.

This could be a task for a follow-up PR, but very interested to hear other's opinions on this 🙂

Comment on lines 30 to 38
TEST_HADOOP("testhadoop", SparkCatalog.class.getName(), ImmutableMap.of(
"type", "hadoop"
)),
SPARK_CATALOG("spark_catalog", SparkSessionCatalog.class.getName(), ImmutableMap.of(
"type", "hive",
"default-namespace", "default",
"parquet-enabled", "true",
"cache-enabled", "false" // Spark will delete tables using v1, leaving the cache out of sync
));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we had constructors that provided a copy or something, so we had SparkCatalogType.testHive(ImmutableMap.of("additionalConfig1", "additionalValue1"), this would have much more benefit for the tests that need to override the Parameters to specify different catalog configuration.

Though that could be a v2 task.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for your reply.
Do you mean that the developers to specify only the additional config, without having to redeclare the entire catalog repeatedly?

SparkSpecifyCatalogTestBase provides the constructor as shown below, it can be used to add additional configuration items.

public SparkSpecifyCatalogTestBase(SparkCatalogType sparkCatalogType, Map<String, String> config) {
this.implementation = sparkCatalogType.getImplementation();
this.catalogConfig = new HashMap<>(sparkCatalogType.getConfig());
if (config != null && !config.isEmpty()) {
config.forEach((key, value) -> catalogConfig.merge(key, value, (oldValue, newValue) -> newValue));
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

super(SparkCatalogType.SPARK_CATALOG, ImmutableMap.of("additionalConfig1", "additionalValue1");
Is it possible to use this method to achieve the desired results?

@hililiwei hililiwei force-pushed the ICEBERG-3499 branch 2 times, most recently from 02a32a6 to 603808b Compare November 15, 2021 06:26
import java.util.Map;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;

public enum SparkCatalogType {
Copy link
Contributor

@rdblue rdblue Nov 16, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wouldn't say this is a catalog type, it's a catalog configuration. We could add more that have slightly different configs.

Copy link
Contributor Author

@hililiwei hililiwei Nov 17, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We could add more that have slightly different configs.

Could you provide more detailed guidance?I'm not familiar with this part.thank you

public TestRuntimeFiltering(String catalogName, String implementation, Map<String, String> config) {
super(catalogName, implementation, config);
public TestRuntimeFiltering() {
super(SparkCatalogType.SPARK_CATALOG);
Copy link
Contributor

@rdblue rdblue Nov 16, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should use Hadoop for tests that only use one catalog. Also, if we're using Hadoop then we don't need to use the test base that sets up the metastore. Or at least we should signal to that test base that it should not set up the metastore if it won't be used by a catalog.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@BeforeClass
public static void startMetastoreAndSpark() {
SparkTestBase.metastore = new TestHiveMetastore();
metastore.start();
SparkTestBase.hiveConf = metastore.hiveConf();
SparkTestBase.spark = SparkSession.builder()
.master("local[2]")
.config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic")
.config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname))
.enableHiveSupport()
.getOrCreate();
SparkTestBase.catalog = (HiveCatalog)
CatalogUtil.loadCatalog(HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf);
try {
catalog.createNamespace(Namespace.of("default"));
} catch (AlreadyExistsException ignored) {
// the default namespace already exists. ignore the create error
}
}

Can this be achieved by modifying the code as follows?


  @Before
  public void checkMetastoreAndSpark() {
    if (SparkTestBase.spark == null) {
      synchronized (SparkTestBase.class) {
        if (SparkTestBase.spark == null) {
          if (StringUtils.equals(catalogName, SPARK_CATALOG_HADOOP.catalogName())) {
            startSpark();
          } else {
            startMetastoreAndSpark();
          }
        }
      }
    }
  }

  public static void startSpark() {
    SparkTestBase.spark = SparkSession.builder()
        .master("local[2]")
        .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic")
        .getOrCreate();
  }

  public static void startMetastoreAndSpark() {
    SparkTestBase.metastore = new TestHiveMetastore();
    metastore.start();
    SparkTestBase.hiveConf = metastore.hiveConf();

    SparkTestBase.spark = SparkSession.builder()
        .master("local[2]")
        .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic")
        .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname))
        .enableHiveSupport()
        .getOrCreate();

    SparkTestBase.catalog = (HiveCatalog)
        CatalogUtil.loadCatalog(HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf);

    try {
      catalog.createNamespace(Namespace.of("default"));
    } catch (AlreadyExistsException ignored) {
      // the default namespace already exists. ignore the create error
    }
  }

import org.junit.Rule;
import org.junit.rules.TemporaryFolder;

public abstract class SparkSpecifyCatalogTestBase extends SparkTestBase {
Copy link
Contributor

@rdblue rdblue Nov 17, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about SparkTestBaseWithCatalog? I think that is clear what this is for. Then SparkCatalogTestBase can be for testing catalogs with Spark.

"parquet-enabled", "true",
"cache-enabled", "false" // Spark will delete tables using v1, leaving the cache out of sync
)),
SPARK_SESSION_CATALOG_HADOOP("spark_catalog", SparkSessionCatalog.class.getName(), ImmutableMap.of(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We only need to test Hive with the Spark session catalog. We don't recommend using other catalogs with the session catalog unless you really know what you're doing.

@rdblue
Copy link
Contributor

rdblue commented Nov 17, 2021

@hililiwei, I opened a PR against your branch to make this a bit more generic and implement my suggestions.

@hililiwei
Copy link
Contributor Author

@hililiwei, I opened a PR against your branch to make this a bit more generic and implement my suggestions.

Thanks for your guidance. merged.

@hililiwei hililiwei requested a review from rdblue November 18, 2021 09:27
@rdblue rdblue merged commit 419684a into apache:master Nov 18, 2021
@rdblue
Copy link
Contributor

rdblue commented Nov 18, 2021

Thanks, @hililiwei! This should really help us reduce test runtime in CI.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants