[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##
@@ -344,17 +346,18 @@ protected void preCommit(HoodieInstant inflightInstant, 
HoodieCommitMetadata met
 
   /**
* Write the HoodieCommitMetadata to metadata table if available.
-   * @param table {@link HoodieTable} of interest.
+   *
+   * @param table   {@link HoodieTable} of interest.
* @param instantTime instant time of the commit.
-   * @param actionType action type of the commit.
-   * @param metadata instance of {@link HoodieCommitMetadata}.
+   * @param actionType  action type of the commit.
+   * @param metadatainstance of {@link HoodieCommitMetadata}.
*/
-  protected void writeTableMetadata(HoodieTable table, String instantTime, 
String actionType, HoodieCommitMetadata metadata) {
+  protected void writeTableMetadata(HoodieTable table, String instantTime, 
String actionType, HoodieCommitMetadata metadata, HoodieData 
writeStatuses) {
 if (table.isTableServiceAction(actionType, instantTime)) {
-  tableServiceClient.writeTableMetadata(table, instantTime, actionType, 
metadata);
+  tableServiceClient.writeTableMetadata(table, instantTime, actionType, 
metadata, writeStatuses);
 } else {
   context.setJobStatus(this.getClass().getSimpleName(), "Committing to 
metadata table: " + config.getTableName());
-  table.getMetadataWriter(instantTime).ifPresent(w -> 
((HoodieTableMetadataWriter) w).update(metadata, instantTime));
+  table.getMetadataWriter(instantTime).ifPresent(w -> 
((HoodieTableMetadataWriter) w).update(metadata, writeStatuses, instantTime));

Review Comment:
   We need not refactor in this PR but I think we need a better abstraction 
that only encapsulates the key and record location, something like 
https://github.com/apache/hudi/pull/7336/files#diff-fde1e56ecf64ce9a748f23f6d9009ff1dd33098c5f795523e3432a645d26803c
   



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -432,6 +468,7 @@ private boolean initializeFromFilesystem(String 
initializationTime, List

[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -430,53 +395,212 @@ private  boolean 
isCommitRevertedByInFlightAction(
   /**
* Initialize the Metadata Table by listing files and partitions from the 
file system.
*
-   * @param dataMetaClient   - {@code HoodieTableMetaClient} for the 
dataset.
+   * @param initializationTime   - Timestamp to use for the commit
+   * @param partitionsToInit - List of MDT partitions to initialize
* @param inflightInstantTimestamp - Current action instant responsible for 
this initialization
*/
-  private boolean initializeFromFilesystem(HoodieTableMetaClient 
dataMetaClient,
+  private boolean initializeFromFilesystem(String initializationTime, 
List partitionsToInit,
Option 
inflightInstantTimestamp) throws IOException {
 if (anyPendingDataInstant(dataMetaClient, inflightInstantTimestamp)) {
   return false;
 }
 
-String createInstantTime = getInitialCommitInstantTime(dataMetaClient);
-
-initializeMetaClient(DEFAULT_METADATA_POPULATE_META_FIELDS);
-initTableMetadata();
-// if async metadata indexing is enabled,
-// then only initialize files partition as other partitions will be built 
using HoodieIndexer
-List enabledPartitionTypes =  new ArrayList<>();
-if (dataWriteConfig.isMetadataAsyncIndex()) {
-  enabledPartitionTypes.add(MetadataPartitionType.FILES);
-} else {
-  // all enabled ones should be initialized
-  enabledPartitionTypes = this.enabledPartitionTypes;
+// FILES partition is always initialized first
+
ValidationUtils.checkArgument(!partitionsToInit.contains(MetadataPartitionType.FILES)
+|| partitionsToInit.get(0).equals(MetadataPartitionType.FILES), 
"FILES partition should be initialized first: " + partitionsToInit);
+
+metadataMetaClient = initializeMetaClient();
+
+// Get a complete list of files and partitions from the file system or 
from already initialized FILES partition of MDT
+boolean filesPartitionAvailable = 
dataMetaClient.getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.FILES);
+List partitionInfoList = filesPartitionAvailable ? 
listAllPartitionsFromMDT(initializationTime) : 
listAllPartitionsFromFilesystem(initializationTime);
+Map> partitionToFilesMap = 
partitionInfoList.stream()
+.map(p -> {
+  String partitionName = 
HoodieTableMetadataUtil.getPartitionIdentifier(p.getRelativePath());
+  return Pair.of(partitionName, p.getFileNameToSizeMap());
+})
+.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
+
+for (MetadataPartitionType partitionType : partitionsToInit) {
+  // Find the commit timestamp to use for this partition. Each 
initialization should use its own unique commit time.
+  String commitTimeForPartition = 
generateUniqueCommitInstantTime(initializationTime);
+
+  LOG.info("Initializing MDT partition " + partitionType + " at instant " 
+ commitTimeForPartition);
+
+  Pair> fileGroupCountAndRecordsPair;
+  switch (partitionType) {
+case FILES:
+  fileGroupCountAndRecordsPair = 
initializeFilesPartition(initializationTime, partitionInfoList);
+  break;
+case BLOOM_FILTERS:
+  fileGroupCountAndRecordsPair = 
initializeBloomFiltersPartition(initializationTime, partitionToFilesMap);
+  break;
+case COLUMN_STATS:
+  fileGroupCountAndRecordsPair = 
initializeColumnStatsPartition(partitionToFilesMap);
+  break;
+case RECORD_INDEX:
+  fileGroupCountAndRecordsPair = initializeRecordIndexPartition();
+  break;
+default:
+  throw new HoodieMetadataException("Unsupported MDT partition type: " 
+ partitionType);
+  }
+
+  // Generate the file groups
+  final int fileGroupCount = fileGroupCountAndRecordsPair.getKey();
+  ValidationUtils.checkArgument(fileGroupCount > 0, "FileGroup count for 
MDT partition " + partitionType + " should be > 0");
+  initializeFileGroups(dataMetaClient, partitionType, 
commitTimeForPartition, fileGroupCount);
+
+  // Perform the commit using bulkCommit
+  HoodieData records = 
fileGroupCountAndRecordsPair.getValue();
+  bulkCommit(commitTimeForPartition, partitionType, records, 
fileGroupCount);
+  metadataMetaClient.reloadActiveTimeline();
+  
dataMetaClient.getTableConfig().setMetadataPartitionState(dataMetaClient, 
partitionType, true);
+  initMetadataReader();
 }
-initializeEnabledFileGroups(dataMetaClient, createInstantTime, 
enabledPartitionTypes);
-initialCommit(createInstantTime, enabledPartitionTypes);
-updateInitializedPartitionsInTableConfig(enabledPartitionTypes);
+

[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -557,32 +558,33 @@ private HoodieData 
readRecordKeysFromBaseFiles(HoodieEngineContext
 
   final String fileId = FSUtils.getFileId(filename);
   final String instantTime = FSUtils.getCommitTime(filename);
-  HoodieFileReader reader = 
HoodieFileReaderFactory.getReaderFactory(HoodieRecord.HoodieRecordType.AVRO).getFileReader(hadoopConf.get(),
 dataFilePath);
-  Iterator recordKeyIterator = reader.getRecordKeyIterator();
+  try (HoodieFileReader reader = 
HoodieFileReaderFactory.getReaderFactory(HoodieRecord.HoodieRecordType.AVRO).getFileReader(hadoopConf.get(),
 dataFilePath)) {

Review Comment:
   Got you. Thanks for the clarification.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -734,6 +736,7 @@ private void initializeFileGroups(HoodieTableMetaClient 
dataMetaClient, Metadata
   }
 } catch (FileNotFoundException e) {
   // If the partition did not exist yet, it will be created below
+  LOG.warn("Exception seen while removing existing file groups in 
partition {} ", partitionPath.getName(), e);

Review Comment:
   +1 please remove it.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java:
##
@@ -193,121 +190,126 @@ public HoodieData> 
getRecordsByKeyPrefixes(L
   return Collections.emptyIterator();
 }
 
-  boolean fullKeys = false;
+boolean fullKeys = false;
 
-  Map>> 
logRecords =
-  readLogRecords(logRecordScanner, sortedKeyPrefixes, 
fullKeys, timings);
+Map> logRecords =
+readLogRecords(logRecordScanner, sortedKeyPrefixes, 
fullKeys, timings);
 
-  List>>> 
mergedRecords =
-  readFromBaseAndMergeWithLogRecords(baseFileReader, 
sortedKeyPrefixes, fullKeys, logRecords, timings, partitionName);
+Map> mergedRecords 
=
+readFromBaseAndMergeWithLogRecords(baseFileReader, 
sortedKeyPrefixes, fullKeys, logRecords, timings, partitionName);
 
-  LOG.debug(String.format("Metadata read for %s keys took 
[baseFileRead, logMerge] %s ms",
-  sortedKeyPrefixes.size(), timings));
+LOG.debug(String.format("Metadata read for %s keys took 
[baseFileRead, logMerge] %s ms",
+sortedKeyPrefixes.size(), timings));
 
-  return mergedRecords.stream()
-.map(keyRecordPair -> keyRecordPair.getValue().orElse(null))
-.filter(Objects::nonNull)
-.iterator();
-} catch (IOException ioe) {
-  throw new HoodieIOException("Error merging records from metadata 
table for  " + sortedKeyPrefixes.size() + " key : ", ioe);
-} finally {
-  closeReader(readers);
-}
-  });
+return mergedRecords.values().iterator();
+  } catch (IOException ioe) {
+throw new HoodieIOException("Error merging records from 
metadata table for  " + sortedKeyPrefixes.size() + " key : ", ioe);
+  } finally {
+closeReader(readers);
+  }
+});
   }
 
   @Override
-  public List>>> 
getRecordsByKeys(List keys,
-   
   String partitionName) {
-// Sort the columns so that keys are looked up in order
-List sortedKeys = new ArrayList<>(keys);
-Collections.sort(sortedKeys);
-Map, List> partitionFileSliceToKeysMap = 
getPartitionFileSliceToKeysMapping(partitionName, sortedKeys);
-List>>> result = 
new ArrayList<>();
-AtomicInteger fileSlicesKeysCount = new AtomicInteger();
-partitionFileSliceToKeysMap.forEach((partitionFileSlicePair, 
fileSliceKeys) -> {
-  Pair, HoodieMetadataLogRecordReader> readers =
-  getOrCreateReaders(partitionName, partitionFileSlicePair.getRight());
-  try {
-List timings = new ArrayList<>();
-HoodieSeekingFileReader baseFileReader = readers.getKey();
-HoodieMetadataLogRecordReader logRecordScanner = readers.getRight();
-if (baseFileReader == null && logRecordScanner == null) {
-  return;
-}
+  protected Map> 
getRecordsByKeys(List keys, String partitionName) {
+if (keys.isEmpty()) {
+  return Collections.emptyMap();
+}
 
-boolean fullKeys = true;
-Map>> logRecords =
-readLogRecords(logRecordScanner, fileSliceKeys, fullKeys, timings);
-
-result.addAll(readFromBaseAndMergeWithLogRecords(baseFileReader, 
fileSliceKeys, fullKeys, logRecords,
-timings, partitionName));
-
-LOG.debug(String.format("Metadata read for %s keys took [baseFileRead, 
logMerge] %s ms",
-fileSliceKeys.size(), timings));
-fileSlicesKeysCount.addAndGet(fileSliceKeys.size());
-  } catch (IOException ioe) {
-throw new HoodieIOException("Error merging records from metadata table 
for  " + sortedKeys.size() + " key : ", ioe);
-  } finally {
-if (!reuse) {
-  closeReader(readers);
-}
+Map> result;
+
+// Load the file slices for the partition. Each file slice is a shard 
which saves a portion of the keys.
+List partitionFileSlices = 
partitionFileSliceMap.computeIfAbsent(partitionName,
+k -> 
HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, 
metadataFileSystemView, partitionName));
+final int numFileSlices = partitionFileSlices.size();
+ValidationUtils.checkState(numFileSlices > 0, "Number of file slices for 
partition " + partitionName + " should be > 0");
+
+// Lookup keys from each file slice
+if (numFileSlices == 1) {
+  // Optimization for a single slice for smaller metadata table partitions
+  result = lookupKeysFromFileSlice(partitionName, keys, 
partitionFileSlices.get(0));
+} else {
+  // Parallel lookup for large si

[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##
@@ -1620,4 +1620,72 @@ public static String createIndexInitTimestamp(String 
timestamp, int offset) {
   public static String createLogCompactionTimestamp(String timestamp) {
 return timestamp + LOG_COMPACTION_TIMESTAMP_SUFFIX;
   }
+
+  /**
+   * Estimates the file group count to use for a MDT partition.
+   *
+   * @param partitionType Type of the partition for which the file 
group count is to be estimated.
+   * @param recordCount   The number of records expected to be written.
+   * @param averageRecordSize Average size of each record to be writen.
+   * @param minFileGroupCount Minimum number of file groups to use.
+   * @param maxFileGroupCount Maximum number of file groups to use.
+   * @param growthFactor  By what factor are the records (recordCount) 
expected to grow?
+   * @param maxFileGroupSizeBytes Maximum size of the file group.
+   * @return The estimated number of file groups.
+   */
+  public static int estimateFileGroupCount(MetadataPartitionType 
partitionType, long recordCount, int averageRecordSize, int minFileGroupCount,

Review Comment:
   in general, it would be good to have UTs for all public util methods. 
Especially, this one is a critical method for RI. Let's add a UT.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java:
##
@@ -143,46 +126,33 @@ public List getAllPartitionPaths() throws 
IOException {
* @param partitionPath The absolute path of the partition to list
*/
   @Override
-  public FileStatus[] getAllFilesInPartition(Path partitionPath)
-  throws IOException {
-if (isMetadataTableEnabled) {
-  try {
-return fetchAllFilesInPartition(partitionPath);
-  } catch (Exception e) {
-throw new HoodieMetadataException("Failed to retrieve files in 
partition " + partitionPath + " from metadata", e);
-  }
+  public FileStatus[] getAllFilesInPartition(Path partitionPath) throws 
IOException {
+ValidationUtils.checkArgument(isMetadataTableInitialized);
+try {
+  return fetchAllFilesInPartition(partitionPath);
+} catch (Exception e) {
+  throw new HoodieMetadataException("Failed to retrieve files in partition 
" + partitionPath + " from metadata", e);
 }
-
-FileSystemBackedTableMetadata fileSystemBackedTableMetadata =

Review Comment:
   Agree, but this behavior change (not falling back to FS backed metadata) 
caused many tests to fail. I'm fully aligned with the change but it could have 
been done in a separate PR and checked in before RLI. Now, we're pretty close 
to fixing all the tests. But, something to note for all future big 
needle-moving PRs. If there is a breaking change then it's even easier to 
revert.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java:
##
@@ -163,16 +160,16 @@ public List 
getPartitionPathWithPathPrefixes(List relativePathPr
   public HoodieData> 
getRecordsByKeyPrefixes(List keyPrefixes,

  String partitionName,

  boolean shouldLoadInMemory) {
-// Sort the columns so that keys are looked up in order
+// Sort the prefixes so that keys are looked up in order
 List sortedKeyPrefixes = new ArrayList<>(keyPrefixes);
 Collections.sort(sortedKeyPrefixes);
 
 // NOTE: Since we partition records to a particular file-group by full 
key, we will have
 //   to scan all file-groups for all key-prefixes as each of these 
might contain some
 //   records matching the key-prefix
-List partitionFileSlices =
-HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(
-metadataMetaClient, metadataFileSystemView, partitionName);
+List partitionFileSlices = 
partitionFileSliceMap.computeIfAbsent(partitionName,

Review Comment:
   +1 we should fix other partitions in followup 
https://issues.apache.org/jira/browse/HUDI-6398



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java:
##
@@ -143,46 +126,33 @@ public List getAllPartitionPaths() throws 
IOException {
* @param partitionPath The absolute path of the partition to list
*/
   @Override
-  public FileStatus[] getAllFilesInPartition(Path partitionPath)
-  throws IOException {
-if (isMetadataTableEnabled) {
-  try {
-return fetchAllFilesInPartition(partitionPath);
-  } catch (Exception e) {
-throw new HoodieMetadataException("Failed to retrieve files in 
partition " + partitionPath + " from metadata", e);
-  }
+  public FileStatus[] getAllFilesInPartition(Path partitionPath) throws 
IOException {
+ValidationUtils.checkArgument(isMetadataTableInitialized);
+try {
+  return fetchAllFilesInPartition(partitionPath);
+} catch (Exception e) {
+  throw new HoodieMetadataException("Failed to retrieve files in partition 
" + partitionPath + " from metadata", e);
 }
-
-FileSystemBackedTableMetadata fileSystemBackedTableMetadata =

Review Comment:
   Agree, but this behavior change (not falling back to FS backed metadata) 
caused many tests to fail. I'm fully aligned with the change but it could have 
been done in a separate PR and checked in before RLI. 



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java:
##
@@ -135,8 +138,13 @@ protected byte[] serializeRecords(List 
records) throws IOException
   }
 
   final byte[] recordBytes = serializeRecord(record, writerSchema);
-  ValidationUtils.checkState(!sortedRecordsMap.containsKey(recordKey),
-  "Writing multiple records with same key not supported for " + 
this.getClass().getName());
+  if (sortedRecordsMap.containsKey(recordKey)) {
+LOG.error("Found duplicate record with recordKey: " + recordKey);

Review Comment:
   that's right but it's better to drop duplicates.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -432,6 +468,7 @@ private boolean initializeFromFilesystem(String 
initializationTime, List

[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -408,18 +365,32 @@ private boolean initializeFromFilesystem(String 
initializationTime, List> fileGroupCountAndRecordsPair;
-  switch (partitionType) {
-case FILES:
-  fileGroupCountAndRecordsPair = 
initializeFilesPartition(initializationTime, partitionInfoList);
-  break;
-case BLOOM_FILTERS:
-  fileGroupCountAndRecordsPair = 
initializeBloomFiltersPartition(initializationTime, partitionToFilesMap);
-  break;
-case COLUMN_STATS:
-  fileGroupCountAndRecordsPair = 
initializeColumnStatsPartition(partitionToFilesMap);
-  break;
-default:
-  throw new HoodieMetadataException("Unsupported MDT partition type: " 
+ partitionType);
+  try {
+switch (partitionType) {
+  case FILES:
+fileGroupCountAndRecordsPair = 
initializeFilesPartition(partitionInfoList);
+// initialize the metadata reader again so the FILES partition can 
be read
+initMetadataReader();

Review Comment:
   This should be called in the end after 
`dataMetaClient.getTableConfig().setMetadataPartitionState` because 
`HoodieBackedTableMetadata` checks for MDT init based on table config.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -474,14 +511,80 @@ private Pair> 
initializeBloomFiltersPartition(
 return Pair.of(fileGroupCount, records);
   }
 
-  private Pair> 
initializeFilesPartition(String createInstantTime, List 
partitionInfoList) {
+  private Pair> 
initializeRecordIndexPartition() throws IOException {
+final HoodieMetadataFileSystemView fsView = new 
HoodieMetadataFileSystemView(dataMetaClient,

Review Comment:
   makes sense and since this cost would be incurred only during initializing 
RI so it's fine.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -408,18 +365,32 @@ private boolean initializeFromFilesystem(String 
initializationTime, List> fileGroupCountAndRecordsPair;
-  switch (partitionType) {
-case FILES:
-  fileGroupCountAndRecordsPair = 
initializeFilesPartition(initializationTime, partitionInfoList);
-  break;
-case BLOOM_FILTERS:
-  fileGroupCountAndRecordsPair = 
initializeBloomFiltersPartition(initializationTime, partitionToFilesMap);
-  break;
-case COLUMN_STATS:
-  fileGroupCountAndRecordsPair = 
initializeColumnStatsPartition(partitionToFilesMap);
-  break;
-default:
-  throw new HoodieMetadataException("Unsupported MDT partition type: " 
+ partitionType);
+  try {
+switch (partitionType) {
+  case FILES:
+fileGroupCountAndRecordsPair = 
initializeFilesPartition(partitionInfoList);
+// initialize the metadata reader again so the FILES partition can 
be read
+initMetadataReader();

Review Comment:
   This should be called in the end after setting table config because 
`HoodieBackedTableMetadata` checks for MDT init based on table config.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java:
##
@@ -123,6 +124,9 @@ public static HoodieWriteConfig createMetadataWriteConfig(
 .withAllowMultiWriteOnSameInstant(true)
 
.withKeyGenerator(HoodieTableMetadataKeyGenerator.class.getCanonicalName())
 .withPopulateMetaFields(DEFAULT_METADATA_POPULATE_META_FIELDS)
+.withStorageConfig(HoodieStorageConfig.newBuilder()
+
.hfileWriterToAllowDuplicates(writeConfig.hfileWriterToAllowDuplicates())

Review Comment:
   instead of failing, just pick one.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##
@@ -694,17 +695,80 @@ private Long getTableChecksum() {
 return getLong(TABLE_CHECKSUM);
   }
 
-  public List getMetadataPartitionsInflight() {
-return StringUtils.split(
-getStringOrDefault(TABLE_METADATA_PARTITIONS_INFLIGHT, 
StringUtils.EMPTY_STRING),
-CONFIG_VALUES_DELIMITER
-);
+  public Set getMetadataPartitionsInflight() {
+return new HashSet<>(StringUtils.split(
+getStringOrDefault(TABLE_METADATA_PARTITIONS_INFLIGHT, 
StringUtils.EMPTY_STRING),
+CONFIG_VALUES_DELIMITER));
   }
 
   public Set getMetadataPartitions() {
 return new HashSet<>(
-StringUtils.split(getStringOrDefault(TABLE_METADATA_PARTITIONS, 
StringUtils.EMPTY_STRING),
-CONFIG_VALUES_DELIMITER));
+StringUtils.split(getStringOrDefault(TABLE_METADATA_PARTITIONS, 
StringUtils.EMPTY_STRING),
+CONFIG_VALUES_DELIMITER));
+  }
+
+  /**
+   * @returns true if metadata table has been created and is being used for 
this dataset, else returns false.
+   */
+  public boolean isMetadataTableEnabled() {
+return isMetadataPartitionEnabled(MetadataPartitionType.FILES);
+  }
+
+  /**
+   * Checks if metadata table is enabled and the specified partition has been 
initialized.
+   *
+   * @param partition The partition to check
+   * @returns true if the specific partition has been initialized, else 
returns false.
+   */
+  public boolean isMetadataPartitionEnabled(MetadataPartitionType partition) {
+return getMetadataPartitions().contains(partition.getPartitionPath());
+  }
+
+  /**
+   * Enables or disables the specified metadata table partition.
+   *
+   * @param partitionType The partition
+   * @param enabled   If true, the partition is enabled, else disabled
+   */
+  public void setMetadataPartitionState(HoodieTableMetaClient metaClient, 
MetadataPartitionType partitionType, boolean enabled) {
+
ValidationUtils.checkArgument(!partitionType.getPartitionPath().contains(CONFIG_VALUES_DELIMITER),
+"Metadata Table partition path cannot contain a comma: " + 
partitionType.getPartitionPath());
+Set partitions = getMetadataPartitions();
+Set partitionsInflight = getMetadataPartitionsInflight();
+if (enabled) {
+  partitions.add(partitionType.getPartitionPath());
+  partitionsInflight.remove(partitionType.getPartitionPath());
+} else if (partitionType.equals(MetadataPartitionType.FILES)) {
+  // file listing partition is required for all other partitions to work
+  // Disabling file partition will also disable all partitions
+  partitions.clear();
+  partitionsInflight.clear();
+} else {
+  partitions.remove(partitionType.getPartitionPath());
+  partitionsInflight.remove(partitionType.getPartitionPath());
+}
+setValue(TABLE_METADATA_PARTITIONS, 
partitions.stream().sorted().collect(Collectors.joining(CONFIG_VALUES_DELIMITER)));
+setValue(TABLE_METADATA_PARTITIONS_INFLIGHT, 
partitionsInflight.stream().sorted().collect(Collectors.joining(CONFIG_VALUES_DELIMITER)));
+update(metaClient.getFs(), new Path(metaClient.getMetaPath()), getProps());
+LOG.info(String.format("MDT %s partition %s has been %s", 
metaClient.getBasePathV2(), partitionType, enabled ? "enabled" : "disabled"));
+  }
+
+  /**
+   * Enables the specified metadata table partition as inflight.
+   *
+   * @param partitionTypes The list of partitions to enable as inflight.
+   */
+  public void setMetadataPartitionsInflight(HoodieTableMetaClient metaClient, 
List partitionTypes) {

Review Comment:
   https://issues.apache.org/jira/browse/HUDI-6332



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java:
##
@@ -77,7 +79,8 @@ public HoodieCommitMetadata doWriteOperation(String 
commitTime, WriteOperationTy
 HoodieCommitMetadata commitMetadata = super.doWriteOperation(commitTime, 
operationType, newPartitionsToAdd,
 partitionToFilesNameLengthMap, bootstrap, createInflightCommit);
 if (writer != null && !createInflightCommit) {
-  writer.update(commitMetadata, commitTime, false);
+  writer.performTableServices(Option.of(commitTime));
+  writer.update(commitMetadata, 
HoodieListData.eager(Collections.EMPTY_LIST), commitTime);

Review Comment:
   Let's keep it open and track in 
https://issues.apache.org/jira/browse/HUDI-6393
   cc @lokeshj1703 



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java:
##
@@ -170,6 +171,17 @@ public static HoodieWriteConfig createMetadataWriteConfig(
   throw new HoodieMetadataException("Unsupported Metrics Reporter type 
" + writeConfig.getMetricsReporterType());
   }
 }
-return builder.build();
+
+HoodieWriteConfig metadataWriteConfig = builder.build();
+// Inline compaction and auto clean is required as we do not expose this 
table outside
+ValidationUtils.checkArgument(!metadataWriteConfig.isAutoClean(), 
"Cleaning is controlled internally for Metadata table.");

Review Comment:
   Good to have validations for invariants, just in case behavior changes in 
future can be taught in ut/ft.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java:
##
@@ -99,7 +90,25 @@ public interface HoodieTableMetadataWriter extends 
Serializable, AutoCloseable {
* Deletes the given metadata partitions. This path reuses DELETE_PARTITION 
operation.
*
* @param instantTime - instant time when replacecommit corresponding to the 
drop will be recorded in the metadata timeline
-   * @param partitions - list of {@link MetadataPartitionType} to drop
+   * @param partitions  - list of {@link MetadataPartitionType} to drop
*/
   void deletePartitions(String instantTime, List 
partitions);
+
+  /**
+   * It returns write client for metadata table.
+   */
+  BaseHoodieWriteClient getWriteClient();

Review Comment:
   Not yet addressed.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -430,53 +395,212 @@ private  boolean 
isCommitRevertedByInFlightAction(
   /**
* Initialize the Metadata Table by listing files and partitions from the 
file system.
*
-   * @param dataMetaClient   - {@code HoodieTableMetaClient} for the 
dataset.
+   * @param initializationTime   - Timestamp to use for the commit
+   * @param partitionsToInit - List of MDT partitions to initialize
* @param inflightInstantTimestamp - Current action instant responsible for 
this initialization
*/
-  private boolean initializeFromFilesystem(HoodieTableMetaClient 
dataMetaClient,
+  private boolean initializeFromFilesystem(String initializationTime, 
List partitionsToInit,
Option 
inflightInstantTimestamp) throws IOException {
 if (anyPendingDataInstant(dataMetaClient, inflightInstantTimestamp)) {
   return false;
 }
 
-String createInstantTime = getInitialCommitInstantTime(dataMetaClient);
-
-initializeMetaClient(DEFAULT_METADATA_POPULATE_META_FIELDS);
-initTableMetadata();
-// if async metadata indexing is enabled,
-// then only initialize files partition as other partitions will be built 
using HoodieIndexer
-List enabledPartitionTypes =  new ArrayList<>();
-if (dataWriteConfig.isMetadataAsyncIndex()) {
-  enabledPartitionTypes.add(MetadataPartitionType.FILES);
-} else {
-  // all enabled ones should be initialized
-  enabledPartitionTypes = this.enabledPartitionTypes;
+// FILES partition is always initialized first
+
ValidationUtils.checkArgument(!partitionsToInit.contains(MetadataPartitionType.FILES)
+|| partitionsToInit.get(0).equals(MetadataPartitionType.FILES), 
"FILES partition should be initialized first: " + partitionsToInit);
+
+metadataMetaClient = initializeMetaClient();
+
+// Get a complete list of files and partitions from the file system or 
from already initialized FILES partition of MDT
+boolean filesPartitionAvailable = 
dataMetaClient.getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.FILES);
+List partitionInfoList = filesPartitionAvailable ? 
listAllPartitionsFromMDT(initializationTime) : 
listAllPartitionsFromFilesystem(initializationTime);
+Map> partitionToFilesMap = 
partitionInfoList.stream()
+.map(p -> {
+  String partitionName = 
HoodieTableMetadataUtil.getPartitionIdentifier(p.getRelativePath());
+  return Pair.of(partitionName, p.getFileNameToSizeMap());
+})
+.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
+
+for (MetadataPartitionType partitionType : partitionsToInit) {
+  // Find the commit timestamp to use for this partition. Each 
initialization should use its own unique commit time.
+  String commitTimeForPartition = 
generateUniqueCommitInstantTime(initializationTime);
+
+  LOG.info("Initializing MDT partition " + partitionType + " at instant " 
+ commitTimeForPartition);
+
+  Pair> fileGroupCountAndRecordsPair;
+  switch (partitionType) {
+case FILES:
+  fileGroupCountAndRecordsPair = 
initializeFilesPartition(initializationTime, partitionInfoList);
+  break;
+case BLOOM_FILTERS:
+  fileGroupCountAndRecordsPair = 
initializeBloomFiltersPartition(initializationTime, partitionToFilesMap);
+  break;
+case COLUMN_STATS:
+  fileGroupCountAndRecordsPair = 
initializeColumnStatsPartition(partitionToFilesMap);
+  break;
+case RECORD_INDEX:
+  fileGroupCountAndRecordsPair = initializeRecordIndexPartition();
+  break;
+default:
+  throw new HoodieMetadataException("Unsupported MDT partition type: " 
+ partitionType);
+  }
+
+  // Generate the file groups
+  final int fileGroupCount = fileGroupCountAndRecordsPair.getKey();
+  ValidationUtils.checkArgument(fileGroupCount > 0, "FileGroup count for 
MDT partition " + partitionType + " should be > 0");
+  initializeFileGroups(dataMetaClient, partitionType, 
commitTimeForPartition, fileGroupCount);
+
+  // Perform the commit using bulkCommit
+  HoodieData records = 
fileGroupCountAndRecordsPair.getValue();
+  bulkCommit(commitTimeForPartition, partitionType, records, 
fileGroupCount);
+  metadataMetaClient.reloadActiveTimeline();
+  
dataMetaClient.getTableConfig().setMetadataPartitionState(dataMetaClient, 
partitionType, true);
+  initMetadataReader();
 }
-initializeEnabledFileGroups(dataMetaClient, createInstantTime, 
enabledPartitionTypes);
-initialCommit(createInstantTime, enabledPartitionTypes);
-updateInitializedPartitionsInTableConfig(enabledPartitionTypes);
+

[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -111,18 +111,27 @@ public abstract class HoodieBackedTableMetadataWriter 
implements HoodieTableMeta
 
   public static final String METADATA_COMPACTION_TIME_SUFFIX = "001";
 
+  // Virtual keys support for metadata table. This Field is
+  // from the metadata payload schema.
+  private static final String RECORD_KEY_FIELD_NAME = 
HoodieMetadataPayload.KEY_FIELD_NAME;
+
+  // Average size of a record saved within the record index.
+  // Record index has a fixed size schema. This has been calculated based on 
experiments with default settings
+  // for block size (4MB), compression (GZ) and disabling the hudi metadata 
fields.

Review Comment:
   I think we should keep the default same irrespective of MDT partition. But, 
having different block size for each partition is not a bad idea. That will 
depend on the dataset so we cannot decide what's a good default. However, we 
can let users have that choice. We don't have to fix it in this release, but 
let's keep this comment open for tracking - 
https://issues.apache.org/jira/browse/HUDI-6392



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-16 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -111,18 +111,27 @@ public abstract class HoodieBackedTableMetadataWriter 
implements HoodieTableMeta
 
   public static final String METADATA_COMPACTION_TIME_SUFFIX = "001";
 
+  // Virtual keys support for metadata table. This Field is
+  // from the metadata payload schema.
+  private static final String RECORD_KEY_FIELD_NAME = 
HoodieMetadataPayload.KEY_FIELD_NAME;
+
+  // Average size of a record saved within the record index.
+  // Record index has a fixed size schema. This has been calculated based on 
experiments with default settings
+  // for block size (4MB), compression (GZ) and disabling the hudi metadata 
fields.

Review Comment:
   I think we should keep the default same irrespective of MDT partition. But, 
having different block size for each partition is not a bad idea. That will 
depend on the dataset so we cannot decide what's a good default. However, we 
can let users have that choice.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-15 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -147,84 +156,55 @@ protected  
HoodieBackedTableMetadataWriter(Configu
 this.metrics = Option.empty();
 this.enabledPartitionTypes = new ArrayList<>();
 
-if (writeConfig.isMetadataTableEnabled()) {
-  this.tableName = writeConfig.getTableName() + METADATA_TABLE_NAME_SUFFIX;
-  this.metadataWriteConfig = createMetadataWriteConfig(writeConfig, 
failedWritesCleaningPolicy);
-  enabled = true;
-
-  // Inline compaction and auto clean is required as we do not expose this 
table outside
-  ValidationUtils.checkArgument(!this.metadataWriteConfig.isAutoClean(),
-  "Cleaning is controlled internally for Metadata table.");
-  
ValidationUtils.checkArgument(!this.metadataWriteConfig.inlineCompactionEnabled(),
-  "Compaction is controlled internally for metadata table.");
-  // Auto commit is required
-  
ValidationUtils.checkArgument(this.metadataWriteConfig.shouldAutoCommit(),
-  "Auto commit is required for Metadata Table");
-  
ValidationUtils.checkArgument(this.metadataWriteConfig.getWriteStatusClassName().equals(FailOnFirstErrorWriteStatus.class.getName()),
-  "MDT should use " + FailOnFirstErrorWriteStatus.class.getName());
-  // Metadata Table cannot have metadata listing turned on. (infinite 
loop, much?)
-  
ValidationUtils.checkArgument(!this.metadataWriteConfig.isMetadataTableEnabled(),
-  "File listing cannot be used for Metadata Table");
-
-  this.dataMetaClient =
-  
HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(dataWriteConfig.getBasePath()).build();
-  enablePartitions();
-  initRegistry();
-  initialize(engineContext, actionMetadata, inflightInstantTimestamp);
-  initTableMetadata();
-} else {
-  enabled = false;
+this.dataMetaClient = 
HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(dataWriteConfig.getBasePath()).build();
+
+if (dataMetaClient.getTableConfig().isMetadataTableEnabled() || 
writeConfig.isMetadataTableEnabled()) {

Review Comment:
   I'm aligned with the proposal that deletion of index should not be through 
write configs. More akin to the databases. If one does not want to use any 
existing index, then run DROP INDEX. But, I think we should have done this 
behavior change in a separate patch. Some of the tests are probably failing due 
to this change. Keeping the comment open so that we can call it out in the 
release notes.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-12 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java:
##
@@ -143,46 +126,33 @@ public List getAllPartitionPaths() throws 
IOException {
* @param partitionPath The absolute path of the partition to list
*/
   @Override
-  public FileStatus[] getAllFilesInPartition(Path partitionPath)
-  throws IOException {
-if (isMetadataTableEnabled) {
-  try {
-return fetchAllFilesInPartition(partitionPath);
-  } catch (Exception e) {
-throw new HoodieMetadataException("Failed to retrieve files in 
partition " + partitionPath + " from metadata", e);
-  }
+  public FileStatus[] getAllFilesInPartition(Path partitionPath) throws 
IOException {
+ValidationUtils.checkArgument(isMetadataTableInitialized);
+try {
+  return fetchAllFilesInPartition(partitionPath);
+} catch (Exception e) {
+  throw new HoodieMetadataException("Failed to retrieve files in partition 
" + partitionPath + " from metadata", e);
 }
-
-FileSystemBackedTableMetadata fileSystemBackedTableMetadata =

Review Comment:
   I think we should do this change in a separate PR. It is independent of RLI.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-11 Thread via GitHub


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


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataLogRecordReader.java:
##
@@ -106,10 +107,10 @@ public Map> 
getRecordsByKeys(List allRecords = logRecordScanner.getRecords();

Review Comment:
   We were not filtering on `keys` which contain valid partitions. Due to that 
result from this method also had data for `__all_partitions__` key which is not 
needed.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-06-01 Thread via GitHub


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java:
##
@@ -99,7 +90,25 @@ public interface HoodieTableMetadataWriter extends 
Serializable, AutoCloseable {
* Deletes the given metadata partitions. This path reuses DELETE_PARTITION 
operation.
*
* @param instantTime - instant time when replacecommit corresponding to the 
drop will be recorded in the metadata timeline
-   * @param partitions - list of {@link MetadataPartitionType} to drop
+   * @param partitions  - list of {@link MetadataPartitionType} to drop
*/
   void deletePartitions(String instantTime, List 
partitions);
+
+  /**
+   * It returns write client for metadata table.
+   */
+  BaseHoodieWriteClient getWriteClient();

Review Comment:
   rename to `getMetadataTableWriteClient` for clarity?



##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java:
##
@@ -99,7 +90,25 @@ public interface HoodieTableMetadataWriter extends 
Serializable, AutoCloseable {
* Deletes the given metadata partitions. This path reuses DELETE_PARTITION 
operation.
*
* @param instantTime - instant time when replacecommit corresponding to the 
drop will be recorded in the metadata timeline
-   * @param partitions - list of {@link MetadataPartitionType} to drop
+   * @param partitions  - list of {@link MetadataPartitionType} to drop
*/
   void deletePartitions(String instantTime, List 
partitions);
+
+  /**
+   * It returns write client for metadata table.
+   */
+  BaseHoodieWriteClient getWriteClient();
+
+  /**
+   * Returns true if the metadata table is initialized.
+   */
+  boolean isInitialized();

Review Comment:
   Is it needed? Can we not get this from table config? If MDT is initialized 
then we should have some MDT partition as value for 
`hoodie.table.metadata.partitions` or  
`hoodie.table.metadata.partitions.inflight` right?



##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java:
##
@@ -41,35 +44,23 @@ public interface HoodieTableMetadataWriter extends 
Serializable, AutoCloseable {
* @param engineContext
* @param indexPartitionInfos - information about partitions to build such 
as partition type and base instant time
*/
-  void buildMetadataPartitions(HoodieEngineContext engineContext, 
List indexPartitionInfos);
-
-  /**
-   * Initialize file groups for the given metadata partitions when indexing is 
requested.
-   *
-   * @param dataMetaClient - meta client for the data table
-   * @param metadataPartitions - metadata partitions for which file groups 
needs to be initialized
-   * @param instantTime- instant time of the index action
-   * @throws IOException
-   */
-  void initializeMetadataPartitions(HoodieTableMetaClient dataMetaClient, 
List metadataPartitions, String instantTime) throws 
IOException;
+  void buildMetadataPartitions(HoodieEngineContext engineContext, 
List indexPartitionInfos) throws IOException;
 
   /**
* Drop the given metadata partitions.
*
-   * @param metadataPartitions
-   * @throws IOException
+   * @param metadataPartitions List of MDT partitions to drop
+   * @throws IOException on failures
*/
   void dropMetadataPartitions(List metadataPartitions) 
throws IOException;
 
   /**
* Update the metadata table due to a COMMIT operation.
*
-   * @param commitMetadata   commit metadata of the operation of interest.
-   * @param instantTime  instant time of the commit.
-   * @param isTableServiceAction true if caller is a table service. false 
otherwise. Only regular write operations can trigger metadata table services 
and this argument
-   * will assist in this.
+   * @param commitMetadata commit metadata of the operation of interest.
+   * @param instantTimeinstant time of the commit.
*/
-  void update(HoodieCommitMetadata commitMetadata, String instantTime, boolean 
isTableServiceAction);
+  void update(HoodieCommitMetadata commitMetadata, HoodieData 
writeStatuses, String instantTime);

Review Comment:
   Why remove `isTableServiceAction`? Wouldn't we want to distinguish the 
update call due to regular ingestion writer from table service writer?



##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##
@@ -18,14 +18,19 @@
 
 package org.apache.hudi.metadata;
 
+import org.apache.avro.specific.SpecificRecordBase;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;

Review Comment:
   nit: if we can avoid re-ordering imports, it would make review easier. Also, 
I think we put hudi imports first.



##
hudi-client/h

[GitHub] [hudi] codope commented on a diff in pull request #8758: [HUDI-53] Implementation of record_index - a HUDI index based on the metadata table.

2023-05-19 Thread via GitHub


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


##
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java:
##
@@ -254,9 +253,9 @@ public void testMetadataRecordKeyExcludeFromPayload(final 
HoodieTableType tableT
 }, "Metadata table should have valid log files!");
 
 // Verify no base file created yet.
-assertThrows(IllegalStateException.class, () -> {
+/*assertThrows(IllegalStateException.class, () -> {
   verifyMetadataRecordKeyExcludeFromPayloadBaseFiles(table);
-}, "Metadata table should not have a base file yet!");
+}, "Metadata table should not have a base file yet!");*/

Review Comment:
   I noticed that the base file for `files` partition is created. Commented out 
this part for now. Is the expectation that no base file should be created for 
any MDT partition until compaction kicks in?



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org