[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

2022-09-24 Thread GitBox


danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979342473


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.index.bucket;
+
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+super(config);
+bucketRangeStepSize = config.getBucketRangeStepSize();
+  }

Review Comment:
   Log some info about the range bucket just like the base class.



-- 
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] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

2022-09-24 Thread GitBox


danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979342415


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java:
##
@@ -45,12 +48,14 @@ public abstract class HoodieBucketIndex extends 
HoodieIndex {
 
   private static final Logger LOG = 
LogManager.getLogger(HoodieBucketIndex.class);
 
-  protected final int numBuckets;
-  protected final List indexKeyFields;
+  protected int numBuckets;
+  protected List indexKeyFields;
 
   public HoodieBucketIndex(HoodieWriteConfig config) {
 super(config);
-
+if (config.getBucketIndexEngineType() == 
BucketIndexEngineType.RANGE_BUCKET) {
+  return;
+}

Review Comment:
   We should not write sub-class specific logic in base classes. So let's not 
return early here.



-- 
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] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

2022-09-24 Thread GitBox


danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979342023


##
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieRangeBucketIndex.java:
##
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hudi.index.bucket;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.testutils.RawTripTestPayload;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+import org.apache.hudi.table.HoodieSparkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.testutils.HoodieClientTestHarness;
+import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Properties;
+
+import static 
org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestHoodieRangeBucketIndex extends HoodieClientTestHarness {
+
+  private static final Logger LOG = 
LogManager.getLogger(TestHoodieRangeBucketIndex.class);
+  private static final Schema SCHEMA = 
getSchemaFromResource(TestHoodieRangeBucketIndex.class, "/exampleSchema.avsc", 
true);
+  private static final int RANGE_BUCKET_STEP_SIZE = 8;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+initSparkContexts();
+initPath();
+initFileSystem();
+// We have some records to be tagged (two different partitions)
+initMetaClient();
+  }
+
+  @AfterEach
+  public void tearDown() throws Exception {
+cleanupResources();
+  }
+
+  @Test
+  public void testValidateBucketIndexConfig() {
+boolean makeConfigSuccess = false;
+try {
+  makeConfig("_row_key,time");
+  makeConfigSuccess = true;
+} catch (HoodieIndexException e) {
+  LOG.error(e.getMessage());
+  assertFalse(makeConfigSuccess);
+  makeConfig("_row_key");
+  makeConfigSuccess = true;
+}
+assertTrue(makeConfigSuccess);
+  }
+
+  @Test
+  public void testTagLocation() throws Exception {
+long rowKey1 = 1;
+long rowKey2 = 9;
+long rowKey3 = 20;
+String recordStr1 = "{\"_row_key\":\"" + rowKey1 + 
"\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
+String recordStr2 = "{\"_row_key\":\"" + rowKey2 + 
"\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
+String recordStr3 = "{\"_row_key\":\"" + rowKey3 + 
"\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
+String recordStr4 = "{\"_row_key\":\"" + rowKey1 + 
"\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
+RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
+HoodieRecord record1 = new HoodieAvroRecord(
+new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), 
rowChange1);
+RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
+HoodieRecord record2 = new HoodieAvroRecord(
+new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), 
rowChange2);
+RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
+HoodieRecord record3 = new HoodieAvroRecord(
+new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), 
rowChange3);
+RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
+HoodieRecord record4 = new HoodieAvroRecord(
+new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), 
rowChange4);
+JavaRDD> record

[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

2022-09-24 Thread GitBox


danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979341988


##
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieRangeBucketIndex.java:
##
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hudi.index.bucket;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.testutils.RawTripTestPayload;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+import org.apache.hudi.table.HoodieSparkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.testutils.HoodieClientTestHarness;
+import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Properties;
+
+import static 
org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestHoodieRangeBucketIndex extends HoodieClientTestHarness {
+
+  private static final Logger LOG = 
LogManager.getLogger(TestHoodieRangeBucketIndex.class);
+  private static final Schema SCHEMA = 
getSchemaFromResource(TestHoodieRangeBucketIndex.class, "/exampleSchema.avsc", 
true);
+  private static final int RANGE_BUCKET_STEP_SIZE = 8;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+initSparkContexts();
+initPath();
+initFileSystem();
+// We have some records to be tagged (two different partitions)
+initMetaClient();
+  }
+
+  @AfterEach
+  public void tearDown() throws Exception {
+cleanupResources();
+  }
+
+  @Test
+  public void testValidateBucketIndexConfig() {
+boolean makeConfigSuccess = false;
+try {
+  makeConfig("_row_key,time");
+  makeConfigSuccess = true;
+} catch (HoodieIndexException e) {
+  LOG.error(e.getMessage());
+  assertFalse(makeConfigSuccess);
+  makeConfig("_row_key");
+  makeConfigSuccess = true;
+}
+assertTrue(makeConfigSuccess);
+  }
+
+  @Test
+  public void testTagLocation() throws Exception {
+long rowKey1 = 1;
+long rowKey2 = 9;
+long rowKey3 = 20;
+String recordStr1 = "{\"_row_key\":\"" + rowKey1 + 
"\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
+String recordStr2 = "{\"_row_key\":\"" + rowKey2 + 
"\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
+String recordStr3 = "{\"_row_key\":\"" + rowKey3 + 
"\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
+String recordStr4 = "{\"_row_key\":\"" + rowKey1 + 
"\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
+RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
+HoodieRecord record1 = new HoodieAvroRecord(
+new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), 
rowChange1);
+RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
+HoodieRecord record2 = new HoodieAvroRecord(
+new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), 
rowChange2);
+RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
+HoodieRecord record3 = new HoodieAvroRecord(
+new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), 
rowChange3);
+RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
+HoodieRecord record4 = new HoodieAvroRecord(
+new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), 
rowChange4);
+JavaRDD> record

[GitHub] [hudi] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

2022-09-24 Thread GitBox


danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979341624


##
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieRangeBucketIndex.java:
##
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hudi.index.bucket;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.testutils.RawTripTestPayload;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+import org.apache.hudi.table.HoodieSparkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.testutils.HoodieClientTestHarness;
+import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Properties;
+
+import static 
org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestHoodieRangeBucketIndex extends HoodieClientTestHarness {
+
+  private static final Logger LOG = 
LogManager.getLogger(TestHoodieRangeBucketIndex.class);
+  private static final Schema SCHEMA = 
getSchemaFromResource(TestHoodieRangeBucketIndex.class, "/exampleSchema.avsc", 
true);
+  private static final int RANGE_BUCKET_STEP_SIZE = 8;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+initSparkContexts();
+initPath();
+initFileSystem();
+// We have some records to be tagged (two different partitions)
+initMetaClient();
+  }
+
+  @AfterEach
+  public void tearDown() throws Exception {
+cleanupResources();
+  }
+
+  @Test
+  public void testValidateBucketIndexConfig() {
+boolean makeConfigSuccess = false;
+try {
+  makeConfig("_row_key,time");
+  makeConfigSuccess = true;
+} catch (HoodieIndexException e) {
+  LOG.error(e.getMessage());
+  assertFalse(makeConfigSuccess);
+  makeConfig("_row_key");

Review Comment:
   Use assertThrows instead



-- 
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] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

2022-09-24 Thread GitBox


danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979341444


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.index.bucket;
+
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+super(config);
+bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+return false;
+  }
+
+  @Override
+  protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, 
List partitionPath) {
+return new RangeBucketIndexLocationMapper(table, partitionPath);
+  }
+
+  public class RangeBucketIndexLocationMapper implements 
BucketIndexLocationMapper {
+
+/**
+ * Mapping from partitionPath -> bucketId -> fileInfo
+ */
+private final Map> 
partitionPathFileIDList;
+
+public RangeBucketIndexLocationMapper(HoodieTable table, List 
partitions) {
+  partitionPathFileIDList = partitions.stream().collect(Collectors.toMap(p 
-> p, p -> loadPartitionBucketIdFileIdMapping(table, p)));
+}
+
+@Override
+public Option getRecordLocation(HoodieKey key) {
+  int bucketId = BucketIdentifier.getRangeBucketId(key, 
bucketRangeStepSize);
+  Map bucketIdToFileIdMapping = 
partitionPathFileIDList.get(key.getPartitionPath());
+  if (bucketIdToFileIdMapping.containsKey(bucketId)) {
+return Option.ofNullable(bucketIdToFileIdMapping.get(bucketId));
+  } else {
+return Option.ofNullable(new HoodieRecordLocation(null, 
BucketIdentifier.bucketIdStr(bucketId)));
+  }

Review Comment:
   what is the `partitionPathFileIDList` used for ? To support changing the 
step size (or buckets) ?



-- 
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] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

2022-09-24 Thread GitBox


danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979341382


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.index.bucket;
+
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+super(config);
+bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+return false;
+  }
+
+  @Override
+  protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, 
List partitionPath) {
+return new RangeBucketIndexLocationMapper(table, partitionPath);
+  }
+
+  public class RangeBucketIndexLocationMapper implements 
BucketIndexLocationMapper {
+
+/**
+ * Mapping from partitionPath -> bucketId -> fileInfo
+ */
+private final Map> 
partitionPathFileIDList;
+
+public RangeBucketIndexLocationMapper(HoodieTable table, List 
partitions) {
+  partitionPathFileIDList = partitions.stream().collect(Collectors.toMap(p 
-> p, p -> loadPartitionBucketIdFileIdMapping(table, p)));
+}
+
+@Override
+public Option getRecordLocation(HoodieKey key) {
+  int bucketId = BucketIdentifier.getRangeBucketId(key, 
bucketRangeStepSize);
+  Map bucketIdToFileIdMapping = 
partitionPathFileIDList.get(key.getPartitionPath());
+  if (bucketIdToFileIdMapping.containsKey(bucketId)) {
+return Option.ofNullable(bucketIdToFileIdMapping.get(bucketId));
+  } else {
+return Option.ofNullable(new HoodieRecordLocation(null, 
BucketIdentifier.bucketIdStr(bucketId)));
+  }

Review Comment:
   Be caution the instantTime in record lcoation should never be null.



-- 
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] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

2022-09-24 Thread GitBox


danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979341202


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.index.bucket;
+
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Range Bucket indexing mechanism.
+ */
+public class HoodieRangeBucketIndex extends HoodieBucketIndex {
+
+  private final int bucketRangeStepSize;
+
+  public HoodieRangeBucketIndex(HoodieWriteConfig config) {
+super(config);
+bucketRangeStepSize = config.getBucketRangeStepSize();
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+return false;
+  }

Review Comment:
   I guess the log files can also be indexed here ?



-- 
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] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

2022-09-24 Thread GitBox


danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979340081


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIdentifier.java:
##
@@ -42,6 +50,20 @@ public static int getBucketId(HoodieKey hoodieKey, String 
indexKeyFields, int nu
 return (getHashKeys(hoodieKey, indexKeyFields).hashCode() & 
Integer.MAX_VALUE) % numBuckets;
   }
 
+  public static int getRangeBucketNum(String recordKey, int 
bucketRangeStepSize) {
+int index = recordKey.indexOf(":");
+if (index >= 0) {
+  recordKey = recordKey.substring(index + 1);
+}
+try {
+  int bucketNum = (int) (Long.parseLong(recordKey) / bucketRangeStepSize);
+  return bucketNum;
+} catch (Exception e) {
+  LOG.error("RANGE_BUCKET index need a primary key like Interger.", e);
+  throw e;

Review Comment:
   `need a primary key like Interger` -> `needs integer type primary key`



-- 
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] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

2022-09-24 Thread GitBox


danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979338948


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##
@@ -1648,6 +1648,10 @@ public int getBucketIndexNumBuckets() {
 return getIntOrDefault(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS);
   }
 
+  public int getBucketRangeStepSize() {
+return getIntOrDefault(HoodieIndexConfig.RANGE_BUCKET_STEP_SIZE);
+  }

Review Comment:
   `getBucketRangeStepSize` -> `getRangeBucketStepSize`



-- 
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] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

2022-09-24 Thread GitBox


danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r979338893


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java:
##
@@ -674,6 +685,12 @@ public EngineType getEngineType() {
 
 private void validateBucketIndexConfig() {
   if 
(hoodieIndexConfig.getString(INDEX_TYPE).equalsIgnoreCase(HoodieIndex.IndexType.BUCKET.toString()))
 {
+if 
(HoodieIndex.BucketIndexEngineType.RANGE_BUCKET.toString().equalsIgnoreCase(hoodieIndexConfig.getString(BUCKET_INDEX_ENGINE_TYPE)))
 {
+  if 
(hoodieIndexConfig.getStringOrDefault(KeyGeneratorOptions.RECORDKEY_FIELD_NAME).contains(","))
 {
+throw new HoodieIndexException("Range Bucket index only support 
single-record-key. Best auto-increment key.");
+  }
+  return;

Review Comment:
   `single-record-key` -> `single record key field`



-- 
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] danny0405 commented on a diff in pull request #6636: [HUDI-4824]Add new index RANGE_BUCKET , when primary key is auto-increment like most mysql table

2022-09-12 Thread GitBox


danny0405 commented on code in PR #6636:
URL: https://github.com/apache/hudi/pull/6636#discussion_r969195683


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieRangeBucketIndex.java:
##
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.index.bucket;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.utils.LazyIterableIterator;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.function.SerializableFunction;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.table.HoodieTable;

Review Comment:
   Thanks, can we add some test cases for this new index then ? Also 
@alexeykudinkin , do you have interest on this feature ?



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