Skip to content

Commit

Permalink
Removing class objects comparison using the getName(),getSimpleName()
Browse files Browse the repository at this point in the history
Changes to correct Operator Type
  • Loading branch information
bibith4 authored and yingsu00 committed Nov 13, 2024
1 parent 0352ee1 commit 56dcdbb
Show file tree
Hide file tree
Showing 9 changed files with 15 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,9 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.common.JavaUtils;
import org.apache.hadoop.hive.ql.exec.Utilities;
import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
import org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat;
import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
import org.apache.hadoop.hive.serde2.AbstractSerDe;
Expand Down Expand Up @@ -442,8 +444,7 @@ public static long parseHiveTimestamp(String value, DateTimeZone timeZone)

public static boolean isSplittable(InputFormat<?, ?> inputFormat, FileSystem fileSystem, Path path)
{
if ("OrcInputFormat".equals(inputFormat.getClass().getSimpleName()) ||
"RCFileInputFormat".equals(inputFormat.getClass().getSimpleName())) {
if (inputFormat instanceof OrcInputFormat || inputFormat instanceof RCFileInputFormat) {
return true;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -388,11 +388,11 @@ else if (operatorType.equals(TableScanOperator.class.getSimpleName()) || operato

if (plan.isOutputTableWriterFragment()) {
writtenOutputPositions += stageExecutionStats.getOperatorSummaries().stream()
.filter(stats -> stats.getOperatorType().equals(TableWriterOperator.class.getSimpleName()))
.filter(stats -> stats.getOperatorType().equals(TableWriterOperator.OPERATOR_TYPE))
.mapToLong(OperatorStats::getInputPositions)
.sum();
writtenOutputLogicalDataSize += stageExecutionStats.getOperatorSummaries().stream()
.filter(stats -> stats.getOperatorType().equals(TableWriterOperator.class.getSimpleName()))
.filter(stats -> stats.getOperatorType().equals(TableWriterOperator.OPERATOR_TYPE))
.mapToLong(stats -> stats.getInputDataSize().toBytes())
.sum();
writtenOutputPhysicalDataSize += stageExecutionStats.getPhysicalWrittenDataSize().toBytes();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,7 @@ public class TaskExecutor
// Interrupt a split if it is running longer than this AND it's blocked on something known
private static final Predicate<List<StackTraceElement>> DEFAULT_INTERRUPTIBLE_SPLIT_PREDICATE = elements ->
elements.stream()
.anyMatch(element -> element.getClassName().equals(JoniRegexpFunctions.class.getName()));
.anyMatch(element -> element.getClassName().equals(JoniRegexpFunctions.CLASS_NAME));
private static final Duration DEFAULT_INTERRUPT_SPLIT_INTERVAL = new Duration(60, SECONDS);

private static final AtomicLong NEXT_RUNNER_ID = new AtomicLong();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,8 @@
public class AggregationOperator
implements Operator
{
public static final String OPERATOR_TYPE = "AggregationOperator";

public static class AggregationOperatorFactory
implements OperatorFactory
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ public class HashAggregationOperator
implements Operator
{
private static final double MERGE_WITH_MEMORY_RATIO = 0.9;
public static final String OPERATOR_TYPE = "HashAggregationOperator";

public static class HashAggregationOperatorFactory
implements OperatorFactory
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ public class HashBuilderOperator
implements Operator
{
private static final Logger log = Logger.get(HashBuilderOperator.class);
public static final String OPERATOR_TYPE = "HashBuilderOperator";

public static class HashBuilderOperatorFactory
implements OperatorFactory
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@
public class TableWriterOperator
implements Operator
{
public static final String OPERATOR_TYPE = "TableWriterOperator";
public static class TableWriterOperatorFactory
implements OperatorFactory
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -717,7 +717,7 @@ private Optional<String> getAdditionalOperatorInfo(OperatorContext context)
return Optional.empty();
}

if (context.getOperatorType().equals(HashBuilderOperator.class.getSimpleName())) {
if (context.getOperatorType().equals(HashBuilderOperator.OPERATOR_TYPE)) {
Optional<JoinNode> planNode = findPlanNode(context.getPlanNodeId(), JoinNode.class);
if (!planNode.isPresent()) {
return Optional.empty();
Expand All @@ -729,8 +729,8 @@ private Optional<String> getAdditionalOperatorInfo(OperatorContext context)
return Optional.of(info);
}

if (context.getOperatorType().equals(HashAggregationOperator.class.getSimpleName()) ||
context.getOperatorType().equals(AggregationOperator.class.getSimpleName())) {
if (context.getOperatorType().equals(HashAggregationOperator.OPERATOR_TYPE) ||
context.getOperatorType().equals(AggregationOperator.OPERATOR_TYPE)) {
Optional<AggregationNode> planNode = findPlanNode(context.getPlanNodeId(), AggregationNode.class);
if (!planNode.isPresent()) {
return Optional.empty();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@

public final class JoniRegexpFunctions
{
public static final String CLASS_NAME = "JoniRegexpFunctions";
private static final Block EMPTY_BLOCK = VarcharType.VARCHAR.createBlockBuilder(null, 0).build();
private JoniRegexpFunctions()
{
Expand Down

0 comments on commit 56dcdbb

Please sign in to comment.