You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@phoenix.apache.org by GitBox <gi...@apache.org> on 2020/03/13 18:20:11 UTC

[GitHub] [phoenix] kadirozde commented on a change in pull request #733: PHOENIX-5732: Implement starttime, endtime in IndexTool for rebuild a…

kadirozde commented on a change in pull request #733: PHOENIX-5732: Implement starttime, endtime in IndexTool for rebuild a…
URL: https://github.com/apache/phoenix/pull/733#discussion_r392396308
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
 ##########
 @@ -801,40 +741,137 @@ public int run(String[] args) throws Exception {
             }
         } catch (Exception ex) {
             LOGGER.error("An exception occurred while performing the indexing job: "
-                    + ExceptionUtils.getMessage(ex) + " at:\n" + ExceptionUtils.getStackTrace(ex));
+                + ExceptionUtils.getMessage(ex) + " at:\n" + ExceptionUtils.getStackTrace(ex));
             return -1;
-        } finally {
-            boolean rethrowException = false;
-            try {
-                if (connection != null) {
-                    try {
-                        connection.close();
-                    } catch (SQLException e) {
-                        LOGGER.error("Failed to close connection ", e);
-                        rethrowException = true;
-                    }
-                }
-                if (htable != null) {
-                    try {
-                        htable.close();
-                    } catch (IOException e) {
-                        LOGGER.error("Failed to close htable ", e);
-                        rethrowException = true;
-                    }
-                }
-                if (jobFactory != null) {
-                    try {
-                        jobFactory.closeConnection();
-                    } catch (SQLException e) {
-                        LOGGER.error("Failed to close jobFactory ", e);
-                        rethrowException = true;
-                    }
-                }
-            } finally {
-                if (rethrowException) {
-                    throw new RuntimeException("Failed to close resource");
-                }
-            }
+        }
+    }
+
+    private Configuration getConfiguration(String tenantId) {
+        final Configuration configuration = HBaseConfiguration.addHbaseResources(getConf());
+        if (tenantId != null) {
+            configuration.set(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+        }
+        return configuration;
+    }
+
+    private boolean submitIndexToolJob(Connection conn, Configuration configuration)
+            throws Exception {
+        Path outputPath = null;
+        FileSystem fs;
+        if (basePath != null) {
+            outputPath =
+                    CsvBulkImportUtil.getOutputPath(new Path(basePath),
+                            pIndexTable == null ?
+                                    pDataTable.getPhysicalName().getString() :
+                                    pIndexTable.getPhysicalName().getString());
+            fs = outputPath.getFileSystem(configuration);
+            fs.delete(outputPath, true);
+        }
+        JobFactory jobFactory = new JobFactory(conn, configuration, outputPath);
+        job = jobFactory.getJob();
+        if (!isForeground) {
+            LOGGER.info("Running Index Build in Background - Submit async and exit");
+            job.submit();
+            return true;
+        }
+        LOGGER.info("Running Index Build in Foreground. Waits for the build to complete."
+                + " This may take a long time!.");
+        return job.waitForCompletion(true);
+    }
+
+    @VisibleForTesting
+    public void populateIndexToolAttributes(CommandLine cmdLine) {
+        boolean useTenantId = cmdLine.hasOption(TENANT_ID_OPTION.getOpt());
+        boolean useStartTime = cmdLine.hasOption(START_TIME_OPTION.getOpt());
+        boolean useEndTime = cmdLine.hasOption(END_TIME_OPTION.getOpt());
+        boolean verify = cmdLine.hasOption(VERIFY_OPTION.getOpt());
+
+        if (useTenantId) {
+            tenantId = cmdLine.getOptionValue(TENANT_ID_OPTION.getOpt());
+        }
+        if(useStartTime) {
+            startTime = new Long(cmdLine.getOptionValue(START_TIME_OPTION.getOpt()));
+        }
+        if (useEndTime) {
+            endTime = new Long(cmdLine.getOptionValue(END_TIME_OPTION.getOpt()));
+        }
+        if(startTime != null || endTime != null) {
+            validateTimeRange();
+        }
+        if (verify) {
+            String value = cmdLine.getOptionValue(VERIFY_OPTION.getOpt());
+            indexVerifyType = IndexVerifyType.fromValue(value);
+        }
+        schemaName = cmdLine.getOptionValue(SCHEMA_NAME_OPTION.getOpt());
+        dataTable = cmdLine.getOptionValue(DATA_TABLE_OPTION.getOpt());
+        indexTable = cmdLine.getOptionValue(INDEX_TABLE_OPTION.getOpt());
+        isPartialBuild = cmdLine.hasOption(PARTIAL_REBUILD_OPTION.getOpt());
+        qDataTable = SchemaUtil.getQualifiedTableName(schemaName, dataTable);
+        basePath = cmdLine.getOptionValue(OUTPUT_PATH_OPTION.getOpt());
+        isForeground = cmdLine.hasOption(RUN_FOREGROUND_OPTION.getOpt());
+        useSnapshot = cmdLine.hasOption(SNAPSHOT_OPTION.getOpt());
+        shouldDeleteBeforeRebuild = cmdLine.hasOption(DELETE_ALL_AND_REBUILD_OPTION.getOpt());
+    }
+
+    private void validateTimeRange() {
+        Long currentTime = EnvironmentEdgeManager.currentTimeMillis();
+        if (startTime != null && (startTime.compareTo(currentTime) > 0 ||
+                (endTime != null && startTime.compareTo(endTime) >= 0))) {
 
 Review comment:
   We should add a check to make sure the end time is not in the future by comparing it with the current time. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services