yihua commented on code in PR #13927:
URL: https://github.com/apache/hudi/pull/13927#discussion_r2386827431


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java:
##########
@@ -248,7 +248,7 @@ public KeyGenerator createKeyGenerator(TypedProperties 
props) throws IOException
   public ReaderContextFactory<?> getReaderContextFactory(HoodieTableMetaClient 
metaClient) {
     // metadata table reads are only supported by the AvroReaderContext.
     if (metaClient.isMetadataTable()) {
-      return new AvroReaderContextFactory(metaClient);
+      return new AvroReaderContextFactory(metaClient, new TypedProperties());

Review Comment:
   Create a follow-up issue for Flink and Java engine?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java:
##########
@@ -199,6 +200,14 @@ public static HoodieWriteConfig createMetadataWriteConfig(
       properties.put(HoodieMetricsConfig.METRICS_REPORTER_PREFIX.key(),
           writeConfig.getMetricReporterMetricsNamePrefix() + 
METADATA_TABLE_NAME_SUFFIX);
     }
+    // HFile caching properties
+    properties.put(HoodieReaderConfig.HFILE_BLOCK_CACHE_ENABLED.key(), 
writeConfig.getProps()
+        .getOrDefault(HoodieReaderConfig.HFILE_BLOCK_CACHE_ENABLED.key(), 
HoodieReaderConfig.HFILE_BLOCK_CACHE_ENABLED.defaultValue()));
+    properties.put(HoodieReaderConfig.HFILE_BLOCK_CACHE_SIZE.key(), 
writeConfig.getProps()
+        .getOrDefault(HoodieReaderConfig.HFILE_BLOCK_CACHE_SIZE.key(), 
HoodieReaderConfig.HFILE_BLOCK_CACHE_SIZE.defaultValue()));
+    properties.put(HoodieReaderConfig.HFILE_BLOCK_CACHE_TTL_MINUTES.key(), 
writeConfig.getProps()
+        .getOrDefault(HoodieReaderConfig.HFILE_BLOCK_CACHE_TTL_MINUTES.key(), 
HoodieReaderConfig.HFILE_BLOCK_CACHE_TTL_MINUTES.defaultValue()));

Review Comment:
   Use 
`writeConfig.getBooleanOrDefault(HoodieReaderConfig.HFILE_BLOCK_CACHE_ENABLED)`,
 etc.



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java:
##########
@@ -260,7 +260,7 @@ public <I, O> O aggregate(HoodieData<I> data, O zeroValue, 
Functions.Function2<O
   public ReaderContextFactory<?> getReaderContextFactory(HoodieTableMetaClient 
metaClient) {
     // metadata table are only supported by the AvroReaderContext.
     if (metaClient.isMetadataTable()) {
-      return new AvroReaderContextFactory(metaClient);
+      return new AvroReaderContextFactory(metaClient, new TypedProperties());

Review Comment:
   So this, used by the metadata writer, cannot leverage the cache configs?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java:
##########
@@ -199,6 +200,14 @@ public static HoodieWriteConfig createMetadataWriteConfig(
       properties.put(HoodieMetricsConfig.METRICS_REPORTER_PREFIX.key(),
           writeConfig.getMetricReporterMetricsNamePrefix() + 
METADATA_TABLE_NAME_SUFFIX);
     }
+    // HFile caching properties
+    properties.put(HoodieReaderConfig.HFILE_BLOCK_CACHE_ENABLED.key(), 
writeConfig.getProps()
+        .getOrDefault(HoodieReaderConfig.HFILE_BLOCK_CACHE_ENABLED.key(), 
HoodieReaderConfig.HFILE_BLOCK_CACHE_ENABLED.defaultValue()));
+    properties.put(HoodieReaderConfig.HFILE_BLOCK_CACHE_SIZE.key(), 
writeConfig.getProps()
+        .getOrDefault(HoodieReaderConfig.HFILE_BLOCK_CACHE_SIZE.key(), 
HoodieReaderConfig.HFILE_BLOCK_CACHE_SIZE.defaultValue()));
+    properties.put(HoodieReaderConfig.HFILE_BLOCK_CACHE_TTL_MINUTES.key(), 
writeConfig.getProps()
+        .getOrDefault(HoodieReaderConfig.HFILE_BLOCK_CACHE_TTL_MINUTES.key(), 
HoodieReaderConfig.HFILE_BLOCK_CACHE_TTL_MINUTES.defaultValue()));

Review Comment:
   Fixed



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