Skip to content

Conversation

@flyrain
Copy link
Contributor

@flyrain flyrain commented Mar 31, 2022

@github-actions github-actions bot added the MR label Mar 31, 2022
Copy link
Contributor

@anuragmantri anuragmantri left a comment

Choose a reason for hiding this comment

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

If I understand correctly, is the idea here to get snapshot information just by querying the catalog (in cases where we only have access to catalog)?

@flyrain
Copy link
Contributor Author

flyrain commented Apr 1, 2022

If I understand correctly, is the idea here to get snapshot information just by querying the catalog (in cases where we only have access to catalog)?

That's correct.

Copy link
Contributor

@anuragmantri anuragmantri 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 change. It looks harmless to me. I will let others comment.

Copy link
Contributor

@singhpk234 singhpk234 left a comment

Choose a reason for hiding this comment

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

Thanks @flyrain for this feature :) !!!

parameters.put(TableProperties.CURRENT_SNAPSHOT_SUMMARY,
JsonUtil.mapper().writeValueAsString(currentSnapshot.summary()));
} catch (JsonProcessingException e) {
LOG.warn("Failed to convert snapshot summary to a json string", e);
Copy link
Contributor

Choose a reason for hiding this comment

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

[minor] Does it makes sense to capture, snapshotId as well for which we got this exception ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Make sense, will print out snapshot id in the warn message.

parameters.put(TableProperties.CURRENT_SNAPSHOT_TIMESTAMP, String.valueOf(currentSnapshot.timestampMillis()));
try {
parameters.put(TableProperties.CURRENT_SNAPSHOT_SUMMARY,
JsonUtil.mapper().writeValueAsString(currentSnapshot.summary()));
Copy link
Contributor

@singhpk234 singhpk234 Apr 1, 2022

Choose a reason for hiding this comment

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

[question] should we take size of summary, also into consideration, As if when we should keep it or is it always optimal to keep it ?

As per my understanding and I may be wrong here, The scenario I have in mind (not sure how practical it is), If some one want's to track changed partition metrics in summary and snapshot had a lot of changed partition and we set CHANGED_PARTITION_COUNT_PROP to a high value, In this scenario the json size could be very large, which is okayish to keep in File, but in DB I am not sure. Your thoughts ?

setIf(trustPartitionMetrics, builder, CHANGED_PARTITION_COUNT_PROP, changedPartitions.size());
if (trustPartitionMetrics && changedPartitions.size() <= maxChangedPartitionsForSummaries) {
setIf(changedPartitions.size() > 0, builder, PARTITION_SUMMARY_PROP, "true");
for (String key : changedPartitions) {
setIf(key != null, builder, CHANGED_PARTITION_PREFIX + key, partitionSummary(partitionMetrics.get(key)));
}
}

Copy link
Contributor Author

@flyrain flyrain Apr 1, 2022

Choose a reason for hiding this comment

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

Good question! According to https://issues.apache.org/jira/browse/HIVE-12274 and https://issues.apache.org/jira/browse/HIVE-14145, we can at least store 4000 chars as the values before HMS3.x, and more in HMS3.x depending on the under layer RDMS, at least 32672 bytes. I'm going to cap the summary at 4000 chars.

Copy link
Member

@szehon-ho szehon-ho left a comment

Choose a reason for hiding this comment

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

Hi @flyrain I think it is useful for hive-based tooling, thanks for this, had some comments and questions.

Would be nice to get a look from @RussellSpitzer @aokolnychyi @pvary too.

private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
private static final String HIVE_ICEBERG_METADATA_REFRESH_MAX_RETRIES = "iceberg.hive.metadata-refresh-max-retries";
private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS = "iceberg.hive.table-level-lock-evict-ms";
private static final long HIVE_TABLE_PROPERTY_VALUE_SIZE_MAX = 4000;
Copy link
Member

Choose a reason for hiding this comment

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

Should we should expose this configuration, otherwise it may not match what works for all users (different database and backends)? I am not sure what others think.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ideally, it should be configurable. Not sure it's worth to do that at this moment. I'm open to any suggestion.

Copy link
Member

Choose a reason for hiding this comment

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

Are different hms databases an issue? I would hope this would be more or less standardized

Copy link
Member

@szehon-ho szehon-ho Apr 6, 2022

Choose a reason for hiding this comment

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

I guess my comment is not complete, @flyrain's analysis in #4456 (comment) is the exact one. Older versions of Hive had the 4000 limit (due to need to support some older databases), and newer versions increased to 32672.

So, if we don't have some kind of flag, we lose the full potential of current HMS versions just to support the older ones.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It would be nice to rely on a configuration in hive, like MAX_STRING_SIZE, and we can read it from somewhere like hive-site.xml, I didn't find any though. It's a bit weird that we got an Iceberg configuration item indicating the max value size of a HMS, maybe that's fine, the best place to hold it is the catalog config, something like spark.sql.catalog.spark_catalog.max_table_property_size. Does it make sense?

Copy link
Member

Choose a reason for hiding this comment

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

Hm, yea I'm also not entirely sure what the right thing to do here is. Yea it's hidden a bit deeply in the package.jdo file in Hive..

I see we do have some Hive specific configs in this file right (like iceberg.hive.lock-timeout-ms for example), and also ConfigProperties.java? Could it be with that prefix (ie, iceberg.hive.summary.stats.max.size), with default value and upper bound 32672? Or if we don't want to give too much complexity to user, maybe iceberg.hive.legacy.table.parameter.size which is just a boolean controlling whether its 32672 or 4000.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sounds good. The only downside is we cannot separate them when two hive catalogs are configured, but that's probably fine. iceberg.hive.lock-timeout-ms couldn't either.

if (summary.length() <= HIVE_TABLE_PROPERTY_VALUE_SIZE_MAX) {
parameters.put(TableProperties.CURRENT_SNAPSHOT_SUMMARY, summary);
} else {
LOG.warn("Not expose the current snapshot({}) summary in HMS since it exceeds {} characters",
Copy link
Member

Choose a reason for hiding this comment

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

nit: "Not exposing".

}

private void setSnapshotStats(TableMetadata metadata, Map<String, String> parameters) {
Snapshot currentSnapshot = metadata.currentSnapshot();
Copy link
Member

Choose a reason for hiding this comment

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

I think we need a way to clear the properties if they are not there anymore in current snapshot (like original code setHmsProperties). For example if the summary becomes suddenly over the length in new snapshot, then it will not override the old summary and would be wrong.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed in the new commit.


// check whether parameters are in expected state
Map<String, String> parameters = hmsTable.getParameters();
Assert.assertEquals("0", parameters.get(TableProperties.SNAPSHOT_COUNT));
Copy link
Member

Choose a reason for hiding this comment

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

Can we put some error messages in these asserts, like in other tests?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'm not sure it is necessary. The code here can explain itself quite well.

try {
String summary = JsonUtil.mapper().writeValueAsString(currentSnapshot.summary());
if (summary.length() <= HIVE_TABLE_PROPERTY_VALUE_SIZE_MAX) {
parameters.put(TableProperties.CURRENT_SNAPSHOT_SUMMARY, summary);
Copy link
Member

Choose a reason for hiding this comment

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

Also had one concern, there's some repeating information (setHmsTableParameters puts some but not all of the summary). Did we ever consider just inlining these fields from the summary like they did over there?

Copy link
Member

Choose a reason for hiding this comment

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

If it's a lot of properties, the json makes sense, just thinking its a shame there is a bit of repeat.

Copy link
Contributor Author

@flyrain flyrain Apr 5, 2022

Choose a reason for hiding this comment

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

These duplicated properties(numFiles, numRows, totalSize) are handy for existing HMS consumers, since they are hive table properties, the existing tool can use them out-of-box. It's nice to keep them as exception cases. However, the snapshot summary have much more stats. It may not be a good idea to split them into individual HMS table properties. For example, any summary change requires a code change here change in that case.

}

@Test
public void testSnapshotStatsTableProperties() throws Exception {
Copy link
Member

Choose a reason for hiding this comment

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

We should probably have a test with the string exceeding the property size threshold, checking to make sure the behavior is as we expect.

Copy link
Contributor Author

@flyrain flyrain Apr 6, 2022

Choose a reason for hiding this comment

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

Added a test case for that

Copy link
Member

@RussellSpitzer RussellSpitzer left a comment

Choose a reason for hiding this comment

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

I had a few minor comments, but @szehon-ho is the Hive expert so when he is +1 I think this is good to merge

@flyrain
Copy link
Contributor Author

flyrain commented Apr 6, 2022

I had a few minor comments, but @szehon-ho is the Hive expert so when he is +1 I think this is good to merge

Thanks for the review. Let me address some comments before the merge.

private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
private static final String HIVE_ICEBERG_METADATA_REFRESH_MAX_RETRIES = "iceberg.hive.metadata-refresh-max-retries";
private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS = "iceberg.hive.table-level-lock-evict-ms";
private static final String HIVE_LEGACY_TABLE_PARAMETER_SIZE = "iceberg.hive.legacy.table.parameter.size";
Copy link
Contributor Author

Choose a reason for hiding this comment

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

The current name doesn't look like a boolean type. I'm open to suggestion. iceberg.hive.is.legacy.table.parameter.size?

Copy link
Member

Choose a reason for hiding this comment

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

hive.use.legacy.table.parameter.size?

@flyrain
Copy link
Contributor Author

flyrain commented Apr 7, 2022

Talked to @szehon-ho offline. We will give a default value for the HIVE_TABLE_PROPERTY_SIZE, while allowing user to configure it with different number due to a variety of HMS backends.

Oracle DB's varchar2 supports 32767 bytes now, by setting the configuration parameter MAX_STRING_SIZE to EXTENDED. (source)
Postgres supports a max of 1GB for character datatype. (source)
MySQL can support upto 65535 bytes for the entire row. So long as the PARAM_KEY value + PARAM_VALUE is less than 65535, we should be good. (source)
SQL Server's varchar max length is 8000 and can go beyond using "varchar(max)" with the same limitation as MySQL being 65535 bytes for the entire row. (source)
Derby's varchar can be upto 32672 bytes. (source)

https://issues.apache.org/jira/browse/HIVE-12274

Copy link
Member

@szehon-ho szehon-ho left a comment

Choose a reason for hiding this comment

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

@flyrain thanks for changes, looks mostly good, just a few more nits

conf.getInt(HIVE_ICEBERG_METADATA_REFRESH_MAX_RETRIES, HIVE_ICEBERG_METADATA_REFRESH_MAX_RETRIES_DEFAULT);
long tableLevelLockCacheEvictionTimeout =
conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
this.maxHiveTableParameterSize = conf.getLong(HIVE_TABLE_PARAMETER_SIZE_MAX, HIVE_TABLE_PARAMETER_SIZE_MAX_DEFAULT);
Copy link
Member

Choose a reason for hiding this comment

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

Nit: can you move it up one line so not to break flow of the tableLevelLockCacheEvictionTimeout statement?

summary.put(String.valueOf(i), "value");
}
long summarySize = JsonUtil.mapper().writeValueAsString(summary).length();
Assert.assertTrue(summarySize > 4000 && summarySize < 32672);
Copy link
Member

Choose a reason for hiding this comment

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

Nit: i think no need for the upper limit check, right? We are not really saving in Hive.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is to make sure the new limit takes affect.

    conf.set("iceberg.hive.table.parameter.size.max", "4000");

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Add a comment in the code

Copy link
Member

@szehon-ho szehon-ho Apr 7, 2022

Choose a reason for hiding this comment

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

Hm, I meant, can't we just do

Assert.assertTrue("Summary size should be greater than limit", summarySize > 4000).

To me the test is checking whether you can persist something beyond 4000 chars right? The fact that it's below or above 32627 chars should not trigger now that we changed limit to 4000 right? Then, no need for the additional comment, if this is the case. Let me know if I am mistaken though.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The main purpose is to test whether we save the summary in HMS parameters when the size exceeds the limit. Besides, I also want to test if the limit has changed from 32627 to 4000. That's why I check both.

 Assert.assertTrue(summarySize > 4000 && summarySize < 32672);

Copy link
Member

Choose a reason for hiding this comment

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

OK got it.

private static final String HIVE_ICEBERG_METADATA_REFRESH_MAX_RETRIES = "iceberg.hive.metadata-refresh-max-retries";
private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS = "iceberg.hive.table-level-lock-evict-ms";
private static final String HIVE_TABLE_PARAMETER_SIZE_MAX = "iceberg.hive.table.parameter.size.max";
private static final long HIVE_TABLE_PARAMETER_SIZE_MAX_DEFAULT = 32672;
Copy link
Member

Choose a reason for hiding this comment

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

Should we add a comment, giving some reference to when it was changed in Hive?

Copy link
Member

@szehon-ho szehon-ho Apr 7, 2022

Choose a reason for hiding this comment

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

Thanks, nit: how about "iceberg.hive.max.table.parameter.size".

Also, regarding the comment, it's kind of based on backend but Hive had a global limit of 4000 regardless of database to allow it to support across different database, so maybe "For Hive versions below 2.3, max table parameter size is 4000 characters, see https://issues.apache.org/jira/browse/HIVE-12274"?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Made the change

Copy link
Member

@szehon-ho szehon-ho Apr 7, 2022

Choose a reason for hiding this comment

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

one more nit: can the variable match the changed string

Copy link
Member

@szehon-ho szehon-ho Apr 7, 2022

Choose a reason for hiding this comment

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

Actually sorry about this , I just noticed other flags, weird how they do dash after "iceberg.hive". How about HIVE_TABLE_PROPERTY_MAX_SIZE / 'iceberg.hive.table-property-max-size" (And fixes the correct term to be table-property).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Made the change.

@flyrain
Copy link
Contributor Author

flyrain commented Apr 7, 2022

The failed test should not be related.

org.apache.iceberg.flink.TestFlinkUpsert > testPrimaryKeyFieldsAtEndOfTableSchema[catalogName=testhadoop, baseNamespace=default, format=PARQUET, isStreaming=true] FAILED
    java.lang.AssertionError: 
    Expecting:
      [+I[3, bbb, 2022-03-01], +I[1, aaa, 2022-03-01]]
    to contain exactly in any order:
      [+I[2, aaa, 2022-03-01], +I[3, bbb, 2022-03-01]]
    elements not found:
      [+I[2, aaa, 2022-03-01]]
    and elements not expected:
      [+I[1, aaa, 2022-03-01]]
        at org.apache.iceberg.flink.TestHelpers.assertRows(TestHelpers.java:137)
        at org.apache.iceberg.flink.TestFlinkUpsert.testPrimaryKeyFieldsAtEndOfTableSchema(TestFlinkUpsert.java:262)

Copy link
Member

@szehon-ho szehon-ho left a comment

Choose a reason for hiding this comment

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

Looks good to me. Just left one comment if you have time, otherwise it's fine. Probably can restart the test too.

if (summary.length() <= maxHiveTablePropertySize) {
parameters.put(TableProperties.CURRENT_SNAPSHOT_SUMMARY, summary);
} else {
parameters.remove(TableProperties.CURRENT_SNAPSHOT_SUMMARY);
Copy link
Member

Choose a reason for hiding this comment

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

Optional: we can actually put the parameters.remove() for all the properties above the parameters.put(), here we are doing the remove three times.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good idea. Made the change.

@szehon-ho szehon-ho closed this Apr 7, 2022
@szehon-ho szehon-ho reopened this Apr 7, 2022
@szehon-ho szehon-ho merged commit 61f0854 into apache:master Apr 8, 2022
@szehon-ho
Copy link
Member

Merged, thanks @flyrain for the changes

@flyrain
Copy link
Contributor Author

flyrain commented Apr 8, 2022

Thanks @szehon-ho! Thanks all for the review, @singhpk234, @RussellSpitzer, @anuragmantri.

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

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants