You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2017/03/21 03:43:18 UTC

[11/30] kylin git commit: minor, add option to listAllCubingJobs precisely cont.

minor, add option to listAllCubingJobs precisely cont.


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/f8528637
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/f8528637
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/f8528637

Branch: refs/heads/master-hbase0.98
Commit: f8528637c17d0aa5260df13c0340c314a60c89c1
Parents: 242b331
Author: Hongbin Ma <ma...@apache.org>
Authored: Wed Mar 15 14:39:48 2017 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Wed Mar 15 14:39:58 2017 +0800

----------------------------------------------------------------------
 .../kylin/rest/controller/JobController.java    |   2 +-
 .../apache/kylin/rest/service/CubeService.java  |   6 +-
 .../apache/kylin/rest/service/JobService.java   | 118 +++++++------------
 .../kylin/rest/service/JobServiceTest.java      |   3 +-
 4 files changed, 46 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/f8528637/server-base/src/main/java/org/apache/kylin/rest/controller/JobController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/JobController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/JobController.java
index 12f9e2e..6dc7887 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/JobController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/JobController.java
@@ -70,7 +70,7 @@ public class JobController extends BasicController {
         JobTimeFilterEnum timeFilter = JobTimeFilterEnum.getByCode(jobRequest.getTimeFilter());
 
         try {
-            jobInstanceList = jobService.listAllJobs(jobRequest.getCubeName(), jobRequest.getProjectName(), statusList, jobRequest.getLimit(), jobRequest.getOffset(), timeFilter);
+            jobInstanceList = jobService.searchJobs(jobRequest.getCubeName(), jobRequest.getProjectName(), statusList, jobRequest.getLimit(), jobRequest.getOffset(), timeFilter);
         } catch (Exception e) {
             logger.error(e.getLocalizedMessage(), e);
             throw new InternalErrorException(e);

http://git-wip-us.apache.org/repos/asf/kylin/blob/f8528637/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
index ba77f06..a6310d1 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -218,7 +218,7 @@ public class CubeService extends BasicService {
     @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
     public CubeDesc updateCubeAndDesc(CubeInstance cube, CubeDesc desc, String newProjectName, boolean forceUpdate) throws IOException, JobException {
 
-        final List<CubingJob> cubingJobs = jobService.listAllCubingJobs(cube.getName(), null, EnumSet.of(ExecutableState.READY, ExecutableState.RUNNING), true);
+        final List<CubingJob> cubingJobs = jobService.listAllCubingJobs(cube.getName(), null, EnumSet.of(ExecutableState.READY, ExecutableState.RUNNING));
         if (!cubingJobs.isEmpty()) {
             throw new JobException("Cube schema shouldn't be changed with running job.");
         }
@@ -248,7 +248,7 @@ public class CubeService extends BasicService {
 
     @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
     public void deleteCube(CubeInstance cube) throws IOException, JobException {
-        final List<CubingJob> cubingJobs = jobService.listAllCubingJobs(cube.getName(), null, EnumSet.of(ExecutableState.READY, ExecutableState.RUNNING, ExecutableState.ERROR), true);
+        final List<CubingJob> cubingJobs = jobService.listAllCubingJobs(cube.getName(), null, EnumSet.of(ExecutableState.READY, ExecutableState.RUNNING, ExecutableState.ERROR));
         if (!cubingJobs.isEmpty()) {
             throw new JobException("The cube " + cube.getName() + " has running or failed job, please discard it and try again.");
         }
@@ -340,7 +340,7 @@ public class CubeService extends BasicService {
             throw new InternalErrorException("Cube " + cubeName + " doesn't contain any READY segment");
         }
 
-        final List<CubingJob> cubingJobs = jobService.listAllCubingJobs(cube.getName(), null, EnumSet.of(ExecutableState.READY, ExecutableState.RUNNING), true);
+        final List<CubingJob> cubingJobs = jobService.listAllCubingJobs(cube.getName(), null, EnumSet.of(ExecutableState.READY, ExecutableState.RUNNING));
         if (!cubingJobs.isEmpty()) {
             throw new JobException("Enable is not allowed with a running job.");
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/f8528637/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
index 1fafb12..9836766 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
@@ -138,74 +138,6 @@ public class JobService extends BasicService implements InitializingBean {
         }));
     }
 
-    public List<JobInstance> listAllJobs(final String cubeName, final String projectName, final List<JobStatusEnum> statusList, final Integer limitValue, final Integer offsetValue, final JobTimeFilterEnum timeFilter) throws IOException, JobException {
-        Integer limit = (null == limitValue) ? 30 : limitValue;
-        Integer offset = (null == offsetValue) ? 0 : offsetValue;
-        List<JobInstance> jobs = listAllJobs(cubeName, projectName, statusList, timeFilter);
-        Collections.sort(jobs);
-
-        if (jobs.size() <= offset) {
-            return Collections.emptyList();
-        }
-
-        if ((jobs.size() - offset) < limit) {
-            return jobs.subList(offset, jobs.size());
-        }
-
-        return jobs.subList(offset, offset + limit);
-    }
-
-    public List<JobInstance> listAllJobs(final String cubeName, final String projectName, final List<JobStatusEnum> statusList, final JobTimeFilterEnum timeFilter) {
-        Calendar calendar = Calendar.getInstance();
-        calendar.setTime(new Date());
-        long timeStartInMillis = getTimeStartInMillis(calendar, timeFilter);
-        return listCubeJobInstance(cubeName, projectName, statusList, timeStartInMillis, Long.MAX_VALUE);
-    }
-
-    @Deprecated
-    public List<JobInstance> listAllJobs(final String cubeName, final String projectName, final List<JobStatusEnum> statusList, final Integer limitValue, final Integer offsetValue) throws IOException, JobException {
-        Integer limit = (null == limitValue) ? 30 : limitValue;
-        Integer offset = (null == offsetValue) ? 0 : offsetValue;
-        List<JobInstance> jobs = listAllJobs(cubeName, projectName, statusList);
-        Collections.sort(jobs);
-
-        if (jobs.size() <= offset) {
-            return Collections.emptyList();
-        }
-
-        if ((jobs.size() - offset) < limit) {
-            return jobs.subList(offset, jobs.size());
-        }
-
-        return jobs.subList(offset, offset + limit);
-    }
-
-    public List<JobInstance> listAllJobs(final String cubeName, final String projectName, final List<JobStatusEnum> statusList) {
-        return listCubeJobInstance(cubeName, projectName, statusList);
-    }
-
-    private List<JobInstance> listCubeJobInstance(final String cubeName, final String projectName, List<JobStatusEnum> statusList, final long timeStartInMillis, final long timeEndInMillis) {
-        Set<ExecutableState> states = convertStatusEnumToStates(statusList);
-        final Map<String, Output> allOutputs = getExecutableManager().getAllOutputs(timeStartInMillis, timeEndInMillis);
-        return Lists.newArrayList(FluentIterable.from(listAllCubingJobs(cubeName, projectName, states, timeStartInMillis, timeEndInMillis, allOutputs, false)).transform(new Function<CubingJob, JobInstance>() {
-            @Override
-            public JobInstance apply(CubingJob cubingJob) {
-                return parseToJobInstance(cubingJob, allOutputs);
-            }
-        }));
-    }
-
-    private List<JobInstance> listCubeJobInstance(final String cubeName, final String projectName, List<JobStatusEnum> statusList) {
-        Set<ExecutableState> states = convertStatusEnumToStates(statusList);
-        final Map<String, Output> allOutputs = getExecutableManager().getAllOutputs();
-        return Lists.newArrayList(FluentIterable.from(listAllCubingJobs(cubeName, projectName, states, allOutputs, false)).transform(new Function<CubingJob, JobInstance>() {
-            @Override
-            public JobInstance apply(CubingJob cubingJob) {
-                return parseToJobInstance(cubingJob, allOutputs);
-            }
-        }));
-    }
-
     private Set<ExecutableState> convertStatusEnumToStates(List<JobStatusEnum> statusList) {
         Set<ExecutableState> states;
         if (statusList == null || statusList.isEmpty()) {
@@ -484,11 +416,44 @@ public class JobService extends BasicService implements InitializingBean {
         return job;
     }
 
-    public List<CubingJob> listAllCubingJobs(final String cubeName, final String projectName, final Set<ExecutableState> statusList, final Map<String, Output> allOutputs, final boolean bEqual) {
-        return listAllCubingJobs(cubeName, projectName, statusList, 0L, Long.MAX_VALUE, allOutputs, bEqual);
+    /**
+     * currently only support substring match
+     * @return
+     */
+    public List<JobInstance> searchJobs(final String cubeNameSubstring, final String projectName, final List<JobStatusEnum> statusList, final Integer limitValue, final Integer offsetValue, final JobTimeFilterEnum timeFilter) throws IOException, JobException {
+        Integer limit = (null == limitValue) ? 30 : limitValue;
+        Integer offset = (null == offsetValue) ? 0 : offsetValue;
+        List<JobInstance> jobs = searchJobs(cubeNameSubstring, projectName, statusList, timeFilter);
+        Collections.sort(jobs);
+
+        if (jobs.size() <= offset) {
+            return Collections.emptyList();
+        }
+
+        if ((jobs.size() - offset) < limit) {
+            return jobs.subList(offset, jobs.size());
+        }
+
+        return jobs.subList(offset, offset + limit);
+    }
+
+    private List<JobInstance> searchJobs(final String cubeNameSubstring, final String projectName, final List<JobStatusEnum> statusList, final JobTimeFilterEnum timeFilter) {
+        Calendar calendar = Calendar.getInstance();
+        calendar.setTime(new Date());
+        long timeStartInMillis = getTimeStartInMillis(calendar, timeFilter);
+
+        long timeEndInMillis = Long.MAX_VALUE;
+        Set<ExecutableState> states = convertStatusEnumToStates(statusList);
+        final Map<String, Output> allOutputs = getExecutableManager().getAllOutputs(timeStartInMillis, timeEndInMillis);
+        return Lists.newArrayList(FluentIterable.from(searchCubingJobs(cubeNameSubstring, projectName, states, timeStartInMillis, timeEndInMillis, allOutputs, false)).transform(new Function<CubingJob, JobInstance>() {
+            @Override
+            public JobInstance apply(CubingJob cubingJob) {
+                return parseToJobInstance(cubingJob, allOutputs);
+            }
+        }));
     }
 
-    public List<CubingJob> listAllCubingJobs(final String cubeName, final String projectName, final Set<ExecutableState> statusList, long timeStartInMillis, long timeEndInMillis, final Map<String, Output> allOutputs, final boolean bEqual) {
+    public List<CubingJob> searchCubingJobs(final String cubeName, final String projectName, final Set<ExecutableState> statusList, long timeStartInMillis, long timeEndInMillis, final Map<String, Output> allOutputs, final boolean cubeNameExactMatch) {
         List<CubingJob> results = Lists.newArrayList(FluentIterable.from(getExecutableManager().getAllAbstractExecutables(timeStartInMillis, timeEndInMillis, CubingJob.class)).filter(new Predicate<AbstractExecutable>() {
             @Override
             public boolean apply(AbstractExecutable executable) {
@@ -499,7 +464,7 @@ public class JobService extends BasicService implements InitializingBean {
                     String executableCubeName = CubingExecutableUtil.getCubeName(executable.getParams());
                     if (executableCubeName == null)
                         return true;
-                    if (bEqual)
+                    if (cubeNameExactMatch)
                         return executableCubeName.equalsIgnoreCase(cubeName);
                     else
                         return executableCubeName.contains(cubeName);
@@ -537,15 +502,12 @@ public class JobService extends BasicService implements InitializingBean {
         return results;
     }
 
-    public List<CubingJob> listAllCubingJobs(final String cubeName, final String projectName, final Set<ExecutableState> statusList, final boolean bEqual) {
-        return listAllCubingJobs(cubeName, projectName, statusList, getExecutableManager().getAllOutputs(), bEqual);
-    }
-
     public List<CubingJob> listAllCubingJobs(final String cubeName, final String projectName, final Set<ExecutableState> statusList) {
-        return listAllCubingJobs(cubeName, projectName, statusList, getExecutableManager().getAllOutputs(), false);
+        return searchCubingJobs(cubeName, projectName, statusList, 0L, Long.MAX_VALUE, getExecutableManager().getAllOutputs(), true);
     }
 
     public List<CubingJob> listAllCubingJobs(final String cubeName, final String projectName) {
-        return listAllCubingJobs(cubeName, projectName, EnumSet.allOf(ExecutableState.class), getExecutableManager().getAllOutputs(), false);
+        return searchCubingJobs(cubeName, projectName, EnumSet.allOf(ExecutableState.class), 0L, Long.MAX_VALUE, getExecutableManager().getAllOutputs(), true);
     }
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/f8528637/server/src/test/java/org/apache/kylin/rest/service/JobServiceTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/kylin/rest/service/JobServiceTest.java b/server/src/test/java/org/apache/kylin/rest/service/JobServiceTest.java
index 0493be6..4150808 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/JobServiceTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/JobServiceTest.java
@@ -20,6 +20,7 @@ package org.apache.kylin.rest.service;
 
 import java.io.IOException;
 
+import org.apache.kylin.job.constant.JobTimeFilterEnum;
 import org.apache.kylin.job.exception.JobException;
 import org.apache.kylin.metadata.project.ProjectInstance;
 import org.junit.Assert;
@@ -44,6 +45,6 @@ public class JobServiceTest extends ServiceTestBase {
         Assert.assertNotNull(jobService.getMetadataManager());
         Assert.assertNotNull(cacheService.getOLAPDataSource(ProjectInstance.DEFAULT_PROJECT_NAME));
         Assert.assertNull(jobService.getJobInstance("job_not_exist"));
-        Assert.assertNotNull(jobService.listAllJobs(null, null, null));
+        Assert.assertNotNull(jobService.searchJobs(null, null, null, 0, 0, JobTimeFilterEnum.ALL));
     }
 }