[GitHub] [hbase] ragarkar commented on a diff in pull request #4799: HBASE-27389 Add cost function in balancer to consider the cost of bui…

2023-03-03 Thread via GitHub


ragarkar commented on code in PR #4799:
URL: https://github.com/apache/hbase/pull/4799#discussion_r1124412915


##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java:
##
@@ -0,0 +1,89 @@
+/*
+ * 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.hadoop.hbase.master.balancer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Compute the cost of a potential cluster configuration based on the number 
of HFile's already
+ * cached in the bucket cache
+ */
+@InterfaceAudience.Private
+public class PrefetchCacheCostFunction extends CostFunction {
+  private static final String PREFETCH_CACHE_COST_KEY =
+"hbase.master.balancer.stochastic.prefetchCacheCost";
+  private static final float DEFAULT_PREFETCH_COST = 500;
+
+  private String prefetchedFileListPath;
+  private double prefetchRatio;
+  private float bestPrefetchRatio;
+
+  /**
+   * Enables or disables the prefetch cache cost function depending on the 
parameter
+   * PREFETCH_PERSISTENCE_PATH_KEY. If set, this parameter enables the 
prefetched file list
+   * persistence. If this parameter is not set this means that the cache 
persistence is disabled
+   * which means that the prefetch ratios of regions on region servers cannot 
be calculated and
+   * hence the regions should be moved based on how much they have been 
prefetched on a region
+   * server. The prefetch cache cost function is disabled if the multiplier is 
set to 0.

Review Comment:
   Done. Made the suggested changes to javadoc.



-- 
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: issues-unsubscr...@hbase.apache.org

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



[GitHub] [hbase] ragarkar commented on a diff in pull request #4799: HBASE-27389 Add cost function in balancer to consider the cost of bui…

2023-03-02 Thread via GitHub


ragarkar commented on code in PR #4799:
URL: https://github.com/apache/hbase/pull/4799#discussion_r1123032103


##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java:
##
@@ -0,0 +1,82 @@
+/*
+ * 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.hadoop.hbase.master.balancer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Compute the cost of a potential cluster configuration based on the number 
of HFile's already
+ * cached in the bucket cache
+ */
+@InterfaceAudience.Private
+public class PrefetchCacheCostFunction extends CostFunction {
+  private static final String PREFETCH_CACHE_COST_KEY =
+"hbase.master.balancer.stochastic.prefetchCacheCost";
+  private static final float DEFAULT_PREFETCH_COST = 500;
+
+  private String prefetchedFileListPath;
+  private double prefetchRatio;
+  private float bestPrefetchRatio;
+
+  PrefetchCacheCostFunction(Configuration conf) {
+prefetchedFileListPath = 
conf.get(HConstants.PREFETCH_PERSISTENCE_PATH_KEY);
+// Disable the prefetch cache cost function if the prefetched file list 
persistence is not
+// enabled
+this.setMultiplier(prefetchedFileListPath == null

Review Comment:
   Done. Added javadoc comment.



-- 
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: issues-unsubscr...@hbase.apache.org

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



[GitHub] [hbase] ragarkar commented on a diff in pull request #4799: HBASE-27389 Add cost function in balancer to consider the cost of bui…

2023-03-02 Thread via GitHub


ragarkar commented on code in PR #4799:
URL: https://github.com/apache/hbase/pull/4799#discussion_r1123031306


##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java:
##
@@ -256,6 +262,7 @@ protected void loadConf(Configuration conf) {
 addCostFunction(new WriteRequestCostFunction(conf));
 addCostFunction(new MemStoreSizeCostFunction(conf));
 addCostFunction(new StoreFileCostFunction(conf));
+addCostFunction(prefetchCacheCost);

Review Comment:
   Done. Added references to prefetchCacheCostFunction to the javadoc.



-- 
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: issues-unsubscr...@hbase.apache.org

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



[GitHub] [hbase] ragarkar commented on a diff in pull request #4799: HBASE-27389 Add cost function in balancer to consider the cost of bui…

2023-03-02 Thread via GitHub


ragarkar commented on code in PR #4799:
URL: https://github.com/apache/hbase/pull/4799#discussion_r1123027979


##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerClusterState.java:
##
@@ -553,6 +569,108 @@ enum LocalityType {
 RACK
   }
 
+  public float getOrComputeWeightedPrefetchRatio(int region, int server) {
+return getRegionSizeMB(region) * getOrComputeRegionPrefetchRatio(region, 
server);
+  }
+
+  protected float getRegionServerPrefetchRatio(int region, int 
regionServerIndex) {
+// Cost this server has from RegionLoad
+float prefetchRatio = 0.0f;
+
+// Get the prefetch ratio if the region is currently hosted on this server
+for (int regionIndex : regionsPerServer[regionServerIndex]) {
+  if (region != regionIndex) {
+continue;
+  }
+  Deque regionLoadList = regionLoads[regionIndex];
+
+  // The region is currently hosted on this region server. Now, get the 
prefetch cache ratio
+  // for this region on this region server
+  prefetchRatio =
+regionLoadList == null ? 0.0f : 
regionLoadList.getLast().getPrefetchCacheRatio();
+
+  return prefetchRatio;
+}
+
+// Region is not currently hosted on this server. Check if the region was 
prefetched on this
+// server earlier. This can happen when the server was shutdown and the 
cache was persisted.
+// Seartch using the index name and server name and not the index id and 
server id as these ids
+// may change when a server is marked as dead or a new server is added.
+String regionNameAsString = regions[region].getRegionNameAsString();
+String serverNameAsString = servers[regionServerIndex].getServerName();
+if (
+  historicalRegionServerPrefetchRatio != null
+&& historicalRegionServerPrefetchRatio.containsKey(regionNameAsString)
+) {
+  Map serverPrefetchRatio =
+historicalRegionServerPrefetchRatio.get(regionNameAsString);
+  if (serverPrefetchRatio.containsKey(serverNameAsString)) {
+prefetchRatio = serverPrefetchRatio.get(serverNameAsString);
+
+// The old prefetch cache ratio has been accounted for and hence, 
clear up this information
+historicalRegionServerPrefetchRatio.remove(regionNameAsString, 
serverPrefetchRatio);
+  }
+}
+return prefetchRatio;
+  }
+
+  private void computeRegionServerPrefetchRatio() {
+regionIndexServerIndexPrefetchRatio = new HashMap<>();
+regionServerIndexWithBestPrefetchRatio = new int[numRegions];
+
+for (int region = 0; region < numRegions; region++) {
+  float bestPrefetchRatio = 0.0f;
+  int serverWithBestPrefetchRatio = 0;
+  for (int server = 0; server < numServers; server++) {
+float prefetchRatio = getRegionServerPrefetchRatio(region, server);
+if (prefetchRatio > 0.0f || server == 
regionIndexToServerIndex[region]) {
+  // A region with prefetch ratio of 0 on a server means nothing. 
Hence, just make a note
+  // of prefetch only if the prefetch ratio is greater than 0.
+  Map tempMap = new HashMap<>();

Review Comment:
   Replaced the Map with Pair. Thanks for the suggestion.



-- 
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: issues-unsubscr...@hbase.apache.org

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



[GitHub] [hbase] ragarkar commented on a diff in pull request #4799: HBASE-27389 Add cost function in balancer to consider the cost of bui…

2023-03-02 Thread via GitHub


ragarkar commented on code in PR #4799:
URL: https://github.com/apache/hbase/pull/4799#discussion_r1123020734


##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java:
##
@@ -452,7 +459,10 @@ protected List balanceTable(TableName 
tableName,
 // Allow turning this feature off if the locality cost is not going to
 // be used in any computations.
 RegionHDFSBlockLocationFinder finder = null;
-if ((this.localityCost != null) || (this.rackLocalityCost != null)) {
+if (
+  (this.localityCost != null) || (this.rackLocalityCost != null)
+|| (this.prefetchCacheCost != null)

Review Comment:
   Removed this code as it 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: issues-unsubscr...@hbase.apache.org

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



[GitHub] [hbase] ragarkar commented on a diff in pull request #4799: HBASE-27389 Add cost function in balancer to consider the cost of bui…

2023-03-01 Thread via GitHub


ragarkar commented on code in PR #4799:
URL: https://github.com/apache/hbase/pull/4799#discussion_r1121338193


##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java:
##
@@ -0,0 +1,82 @@
+/*
+ * 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.hadoop.hbase.master.balancer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Compute the cost of a potential cluster configuration based on the number 
of HFile's already
+ * cached in the bucket cache
+ */
+@InterfaceAudience.Private
+public class PrefetchCacheCostFunction extends CostFunction {
+  private static final String PREFETCH_CACHE_COST_KEY =
+"hbase.master.balancer.stochastic.prefetchCacheCost";
+  private static final float DEFAULT_PREFETCH_COST = 500;
+
+  private String prefetchedFileListPath;
+  private double prefetchRatio;
+  private float bestPrefetchRatio;
+
+  PrefetchCacheCostFunction(Configuration conf) {
+prefetchedFileListPath = 
conf.get(HConstants.PREFETCH_PERSISTENCE_PATH_KEY);
+// Disable the prefetch cache cost function if the prefetched file list 
persistence is not
+// enabled
+this.setMultiplier(prefetchedFileListPath == null
+  ? 0.0f
+  : conf.getFloat(PREFETCH_CACHE_COST_KEY, DEFAULT_PREFETCH_COST));
+prefetchRatio = 0.0f;
+bestPrefetchRatio = 0.0f;
+  }
+
+  @Override
+  void prepare(BalancerClusterState cluster) {
+super.prepare(cluster);
+prefetchRatio = 0.0f;
+bestPrefetchRatio = 0.0f;
+
+for (int region = 0; region < cluster.numRegions; region++) {
+  prefetchRatio +=
+cluster.getOrComputeWeightedPrefetchRatio(region, 
cluster.regionIndexToServerIndex[region]);
+  bestPrefetchRatio += cluster.getOrComputeWeightedPrefetchRatio(region,
+cluster.getOrComputeServerWithBestPrefetchRatio()[region]);
+}
+prefetchRatio = bestPrefetchRatio == 0.0f ? 1.0f : prefetchRatio / 
bestPrefetchRatio;

Review Comment:
   The bestPrefetchRatio could be 0 if a region is not yet hosted on any region 
server which may happen while initialising the cluster.



-- 
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: issues-unsubscr...@hbase.apache.org

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



[GitHub] [hbase] ragarkar commented on a diff in pull request #4799: HBASE-27389 Add cost function in balancer to consider the cost of bui…

2023-02-28 Thread via GitHub


ragarkar commented on code in PR #4799:
URL: https://github.com/apache/hbase/pull/4799#discussion_r1121272883


##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerClusterState.java:
##
@@ -553,6 +569,108 @@ enum LocalityType {
 RACK
   }
 
+  public float getOrComputeWeightedPrefetchRatio(int region, int server) {
+return getRegionSizeMB(region) * getOrComputeRegionPrefetchRatio(region, 
server);
+  }
+
+  protected float getRegionServerPrefetchRatio(int region, int 
regionServerIndex) {
+// Cost this server has from RegionLoad
+float prefetchRatio = 0.0f;
+
+// Get the prefetch ratio if the region is currently hosted on this server
+for (int regionIndex : regionsPerServer[regionServerIndex]) {
+  if (region != regionIndex) {
+continue;
+  }
+  Deque regionLoadList = regionLoads[regionIndex];
+
+  // The region is currently hosted on this region server. Now, get the 
prefetch cache ratio
+  // for this region on this region server
+  prefetchRatio =
+regionLoadList == null ? 0.0f : 
regionLoadList.getLast().getPrefetchCacheRatio();
+
+  return prefetchRatio;
+}
+
+// Region is not currently hosted on this server. Check if the region was 
prefetched on this
+// server earlier. This can happen when the server was shutdown and the 
cache was persisted.
+// Seartch using the index name and server name and not the index id and 
server id as these ids
+// may change when a server is marked as dead or a new server is added.
+String regionNameAsString = regions[region].getRegionNameAsString();
+String serverNameAsString = servers[regionServerIndex].getServerName();
+if (
+  historicalRegionServerPrefetchRatio != null
+&& historicalRegionServerPrefetchRatio.containsKey(regionNameAsString)
+) {
+  Map serverPrefetchRatio =
+historicalRegionServerPrefetchRatio.get(regionNameAsString);
+  if (serverPrefetchRatio.containsKey(serverNameAsString)) {
+prefetchRatio = serverPrefetchRatio.get(serverNameAsString);
+
+// The old prefetch cache ratio has been accounted for and hence, 
clear up this information
+historicalRegionServerPrefetchRatio.remove(regionNameAsString, 
serverPrefetchRatio);
+  }
+}
+return prefetchRatio;

Review Comment:
   This historical prefetch ratio is just an indicator of how much a region was 
prefetched on a region server before it was moved (because of the region server 
going down or some other reason) to another region server. This information is 
mainly used to calculate the difference between the prefetch ratio of this 
region on a new region server and how much this region was prefetched on the 
old region server (if any). If the region's prefetch ratio is higher on the 
currently hosting region server, the region will not be moved back to the old 
region server anyways. However, this historical prefetch ratio is useful if the 
region is only partially prefetched on the new region server while the 
historical prefetch ratio is higher to the extent that it still makes sense to 
move the region back to the old region server assuming that the region's blocks 
were persisted in the cache while shutting down the region server. In absence 
of this metric, it will not be possible to find out how much a re
 gion was historically prefetched on a region server. Without this metric, this 
cost function will never move the regions based on the prefetch ratio as it 
will not have any historical information to compare with.



-- 
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: issues-unsubscr...@hbase.apache.org

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



[GitHub] [hbase] ragarkar commented on a diff in pull request #4799: HBASE-27389 Add cost function in balancer to consider the cost of bui…

2023-02-14 Thread via GitHub


ragarkar commented on code in PR #4799:
URL: https://github.com/apache/hbase/pull/4799#discussion_r1106703131


##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerClusterState.java:
##
@@ -553,6 +558,61 @@ enum LocalityType {
 RACK
   }
 
+  public float getOrComputeWeightedPrefetchRatio(int region, int server) {
+return getRegionSizeMB(region) * 
getOrComputeRegionPrefetchRatio()[region][server];
+  }
+
+  private float[][] getOrComputeRegionPrefetchRatio() {
+if (regionServerWithBestPrefetchRatio == null || regionServerPrefetchRatio 
== null) {
+  computeRegionServerPrefetchRatio();
+}
+return regionServerPrefetchRatio;
+  }
+
+  public int[] getOrComputeServerWithBestPrefetchRatio() {
+if (regionServerWithBestPrefetchRatio == null || regionServerPrefetchRatio 
== null) {
+  computeRegionServerPrefetchRatio();
+}
+return regionServerWithBestPrefetchRatio;
+  }
+
+  private void computeRegionServerPrefetchRatio() {
+regionServerPrefetchRatio = new float[numRegions][numServers];
+regionServerWithBestPrefetchRatio = new int[numRegions];
+
+for (int region = 0; region < numRegions; region++) {
+  float bestPrefetchRatio = 0.0f;
+  int serverWithBestPrefetchRatio = 0;
+  for (int server = 0; server < numServers; server++) {

Review Comment:
   We are iterating through the regions first instead of servers because, we 
also need to find out if the region was previously hosted on that region 
server. If we go the other way i.e. iterate through the servers and then 
iterate through the regions hosted on that server, we will have to iterate 
through all the servers again to find out if this region was previously hosted 
on any of these region servers. Hence, IMO, it's better to iterate through the 
regions before iterating through servers. This way, in the same iteration, we 
can find out whether is region is currently hosted on the given region server 
or if it was previously hosted on this region server.



-- 
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: issues-unsubscr...@hbase.apache.org

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



[GitHub] [hbase] ragarkar commented on a diff in pull request #4799: HBASE-27389 Add cost function in balancer to consider the cost of bui…

2023-02-14 Thread via GitHub


ragarkar commented on code in PR #4799:
URL: https://github.com/apache/hbase/pull/4799#discussion_r1106700557


##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java:
##
@@ -0,0 +1,82 @@
+/*
+ * 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.hadoop.hbase.master.balancer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Compute the cost of a potential cluster configuration based on the number 
of HFile's already
+ * cached in the bucket cache
+ */
+@InterfaceAudience.Private
+public class PrefetchCacheCostFunction extends CostFunction {
+  private static final String PREFETCH_CACHE_COST_KEY =
+"hbase.master.balancer.stochastic.prefetchCacheCost";
+  private static final float DEFAULT_PREFETCH_COST = 500;
+
+  private String prefetchedFileListPath;
+  private double prefetchRatio;
+  private float bestPrefetchRatio;
+
+  PrefetchCacheCostFunction(Configuration conf) {
+prefetchedFileListPath = 
conf.get(HConstants.PREFETCH_PERSISTENCE_PATH_KEY);
+// Disable the prefetch cache cost function if the prefetched file list 
persistence is not
+// enabled
+this.setMultiplier(prefetchedFileListPath == null
+  ? 0.0f
+  : conf.getFloat(PREFETCH_CACHE_COST_KEY, DEFAULT_PREFETCH_COST));
+prefetchRatio = 0.0f;
+bestPrefetchRatio = 0.0f;
+  }
+
+  @Override
+  void prepare(BalancerClusterState cluster) {
+super.prepare(cluster);
+prefetchRatio = 0.0f;
+bestPrefetchRatio = 0.0f;
+
+for (int region = 0; region < cluster.numRegions; region++) {
+  prefetchRatio +=
+cluster.getOrComputeWeightedPrefetchRatio(region, 
cluster.regionIndexToServerIndex[region]);
+  bestPrefetchRatio += cluster.getOrComputeWeightedPrefetchRatio(region,
+cluster.getOrComputeServerWithBestPrefetchRatio()[region]);
+}
+prefetchRatio = bestPrefetchRatio == 0.0f ? 1.0f : prefetchRatio / 
bestPrefetchRatio;

Review Comment:
   The prefetch ratio and the bestPrefetchRatio will be different in case the 
region was hosted previously on some other region server. For example, if the 
region was previously hosted and prefetched 100% on RS-A and later moved to 
RS-B and it was partially prefetched there, the best prefetch ratio will be 
different than the prefetch ratio on individual nodes.



-- 
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: issues-unsubscr...@hbase.apache.org

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



[GitHub] [hbase] ragarkar commented on a diff in pull request #4799: HBASE-27389 Add cost function in balancer to consider the cost of bui…

2023-02-14 Thread via GitHub


ragarkar commented on code in PR #4799:
URL: https://github.com/apache/hbase/pull/4799#discussion_r1106698856


##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java:
##
@@ -0,0 +1,82 @@
+/*
+ * 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.hadoop.hbase.master.balancer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Compute the cost of a potential cluster configuration based on the number 
of HFile's already
+ * cached in the bucket cache
+ */
+@InterfaceAudience.Private
+public class PrefetchCacheCostFunction extends CostFunction {
+  private static final String PREFETCH_CACHE_COST_KEY =
+"hbase.master.balancer.stochastic.prefetchCacheCost";
+  private static final float DEFAULT_PREFETCH_COST = 500;
+
+  private String prefetchedFileListPath;
+  private double prefetchRatio;
+  private float bestPrefetchRatio;
+
+  PrefetchCacheCostFunction(Configuration conf) {
+prefetchedFileListPath = 
conf.get(HConstants.PREFETCH_PERSISTENCE_PATH_KEY);
+// Disable the prefetch cache cost function if the prefetched file list 
persistence is not
+// enabled
+this.setMultiplier(prefetchedFileListPath == null
+  ? 0.0f
+  : conf.getFloat(PREFETCH_CACHE_COST_KEY, DEFAULT_PREFETCH_COST));
+prefetchRatio = 0.0f;
+bestPrefetchRatio = 0.0f;
+  }
+
+  @Override
+  void prepare(BalancerClusterState cluster) {
+super.prepare(cluster);
+prefetchRatio = 0.0f;
+bestPrefetchRatio = 0.0f;
+
+for (int region = 0; region < cluster.numRegions; region++) {
+  prefetchRatio +=
+cluster.getOrComputeWeightedPrefetchRatio(region, 
cluster.regionIndexToServerIndex[region]);
+  bestPrefetchRatio += cluster.getOrComputeWeightedPrefetchRatio(region,
+cluster.getOrComputeServerWithBestPrefetchRatio()[region]);
+}
+prefetchRatio = bestPrefetchRatio == 0.0f ? 1.0f : prefetchRatio / 
bestPrefetchRatio;
+  }
+
+  @Override
+  protected double cost() {
+return 1 - prefetchRatio;
+  }

Review Comment:
   Yes. That is correct. The overall cost is calculated after reviewing all the 
cost functions. If the cost is below a minimum threshold, no action is taken. 
To be in agreement with this, the prefetch based cost function calculates the 
cost this way viz. a higher prefetch means that the cluster is more stable and 
hence, the overall cost of running the cluster in this state is lower.



-- 
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: issues-unsubscr...@hbase.apache.org

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



[GitHub] [hbase] ragarkar commented on a diff in pull request #4799: HBASE-27389 Add cost function in balancer to consider the cost of bui…

2023-02-14 Thread via GitHub


ragarkar commented on code in PR #4799:
URL: https://github.com/apache/hbase/pull/4799#discussion_r1106697411


##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java:
##
@@ -0,0 +1,82 @@
+/*
+ * 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.hadoop.hbase.master.balancer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Compute the cost of a potential cluster configuration based on the number 
of HFile's already
+ * cached in the bucket cache
+ */
+@InterfaceAudience.Private
+public class PrefetchCacheCostFunction extends CostFunction {
+  private static final String PREFETCH_CACHE_COST_KEY =
+"hbase.master.balancer.stochastic.prefetchCacheCost";
+  private static final float DEFAULT_PREFETCH_COST = 500;
+
+  private String prefetchedFileListPath;
+  private double prefetchRatio;
+  private float bestPrefetchRatio;
+
+  PrefetchCacheCostFunction(Configuration conf) {
+prefetchedFileListPath = 
conf.get(HConstants.PREFETCH_PERSISTENCE_PATH_KEY);
+// Disable the prefetch cache cost function if the prefetched file list 
persistence is not
+// enabled
+this.setMultiplier(prefetchedFileListPath == null
+  ? 0.0f
+  : conf.getFloat(PREFETCH_CACHE_COST_KEY, DEFAULT_PREFETCH_COST));
+prefetchRatio = 0.0f;
+bestPrefetchRatio = 0.0f;
+  }
+
+  @Override
+  void prepare(BalancerClusterState cluster) {
+super.prepare(cluster);
+prefetchRatio = 0.0f;
+bestPrefetchRatio = 0.0f;
+
+for (int region = 0; region < cluster.numRegions; region++) {
+  prefetchRatio +=
+cluster.getOrComputeWeightedPrefetchRatio(region, 
cluster.regionIndexToServerIndex[region]);
+  bestPrefetchRatio += cluster.getOrComputeWeightedPrefetchRatio(region,
+cluster.getOrComputeServerWithBestPrefetchRatio()[region]);
+}
+prefetchRatio = bestPrefetchRatio == 0.0f ? 1.0f : prefetchRatio / 
bestPrefetchRatio;
+  }
+
+  @Override
+  protected double cost() {
+return 1 - prefetchRatio;
+  }
+
+  @Override
+  protected void regionMoved(int region, int oldServer, int newServer) {
+float oldServerPrefetch = 
cluster.getOrComputeWeightedPrefetchRatio(region, oldServer);
+float newServerPrefetch = 
cluster.getOrComputeWeightedPrefetchRatio(region, newServer);
+float prefetchDelta = newServerPrefetch - oldServerPrefetch;
+float normalizeDelta = bestPrefetchRatio == 0.0f ? 0.0f : prefetchDelta / 
bestPrefetchRatio;
+prefetchRatio += normalizeDelta;
+  }

Review Comment:
   This code is triggered when the plan calculated by the balancer cannot be 
applied as it will increase the overall cost. In this case, the plan is cleaned 
up and all the updates done to the costs are reverted back through this method 
implemented on each cost function.



-- 
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: issues-unsubscr...@hbase.apache.org

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



[GitHub] [hbase] ragarkar commented on a diff in pull request #4799: HBASE-27389 Add cost function in balancer to consider the cost of bui…

2022-10-05 Thread GitBox


ragarkar commented on code in PR #4799:
URL: https://github.com/apache/hbase/pull/4799#discussion_r985399745


##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java:
##
@@ -0,0 +1,79 @@
+/*
+ * 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.hadoop.hbase.master.balancer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Compute the cost of a potential cluster configuration based on the number 
of HFile's already
+ * cached in the bucket cache
+ */
+@InterfaceAudience.Private
+public class PrefetchCacheCostFunction extends CostFunction {
+  private String prefetchedFileListPath;
+  private float prefetchRatio;
+  private float bestPrefetchRatio;
+
+  public static final String PREFETCH_PERSISTENCE_PATH_KEY = 
"hbase.prefetch.file-list.path";

Review Comment:
   Done. I have made this change. It will be visible in the next patch.



##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java:
##
@@ -0,0 +1,79 @@
+/*
+ * 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.hadoop.hbase.master.balancer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Compute the cost of a potential cluster configuration based on the number 
of HFile's already
+ * cached in the bucket cache
+ */
+@InterfaceAudience.Private
+public class PrefetchCacheCostFunction extends CostFunction {
+  private String prefetchedFileListPath;
+  private float prefetchRatio;
+  private float bestPrefetchRatio;
+
+  public static final String PREFETCH_PERSISTENCE_PATH_KEY = 
"hbase.prefetch.file-list.path";
+
+  PrefetchCacheCostFunction(Configuration conf) {
+prefetchedFileListPath = conf.get(PREFETCH_PERSISTENCE_PATH_KEY);
+this.setMultiplier(prefetchedFileListPath == null ? 0 : 1);

Review Comment:
   Yes. The whole logic of this cost function depends on the capability of 
persisting the list of files already prefetched on the server and hence, if the 
configuration "hbase.prefetch.file-list.path" isn't set, then the cost function 
will be disabled.



##
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java:
##
@@ -452,7 +459,10 @@ protected List balanceTable(TableName 
tableName,
 // Allow turning this feature off if the locality cost is not going to
 // be used in any computations.
 RegionHDFSBlockLocationFinder finder = null;
-if ((this.localityCost != null) || (this.rackLocalityCost != null)) {
+if (
+  (this.localityCost != null) || (this.rackLocalityCost != null)
+|| (this.prefetchCacheCost != null)

Review Comment:
   I have put this code just in case, the localityCost and rackLocalityCost 
functions are disabled by the user configuration?



-- 
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: issues-unsubscr...@hbase.apache.org

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