Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -101,8 +101,8 @@ public int getProgress(InterpreterContext context) {

@Override
public Scheduler getScheduler() {
return SchedulerFactory.singleton().createOrGetFIFOScheduler(
LivyPySparkInterpreter.class.getName() + this.hashCode());
return SchedulerFactory.singleton().createOrGetFIFOPerUserScheduler(
LivySparkInterpreter.class.getName());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.ConcurrentHashMap;

/**
* Livy Spark interpreter for Zeppelin.
Expand All @@ -37,7 +38,7 @@ public class LivySparkInterpreter extends Interpreter {
Logger LOGGER = LoggerFactory.getLogger(LivySparkInterpreter.class);
private LivyOutputStream out;

protected static Map<String, Integer> userSessionMap;
protected static Map<String, Integer> userSessionMap = new ConcurrentHashMap<>();
protected static Map<Integer, String> sessionId2AppIdMap;
protected static Map<Integer, String> sessionId2WebUIMap;

Expand All @@ -46,7 +47,6 @@ public class LivySparkInterpreter extends Interpreter {

public LivySparkInterpreter(Properties property) {
super(property);
userSessionMap = new HashMap<>();
sessionId2AppIdMap = new HashMap<>();
sessionId2WebUIMap = new HashMap<>();
livyHelper = new LivyHelper(property);
Expand Down Expand Up @@ -151,8 +151,8 @@ public int getProgress(InterpreterContext context) {

@Override
public Scheduler getScheduler() {
return SchedulerFactory.singleton().createOrGetFIFOScheduler(
LivySparkInterpreter.class.getName() + this.hashCode());
return SchedulerFactory.singleton().createOrGetFIFOPerUserScheduler(
LivySparkInterpreter.class.getName());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,8 +101,8 @@ public int getProgress(InterpreterContext context) {

@Override
public Scheduler getScheduler() {
return SchedulerFactory.singleton().createOrGetFIFOScheduler(
LivySparkRInterpreter.class.getName() + this.hashCode());
return SchedulerFactory.singleton().createOrGetFIFOPerUserScheduler(
LivySparkInterpreter.class.getName());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
*/
public class LivySparkSQLInterpreter extends Interpreter {

Logger LOGGER = LoggerFactory.getLogger(LivySparkSQLInterpreter.class);
private static final Logger LOGGER = LoggerFactory.getLogger(LivySparkSQLInterpreter.class);

protected Map<String, Integer> userSessionMap;
private LivyHelper livyHelper;
Expand All @@ -57,6 +57,10 @@ public void close() {
@Override
public InterpreterResult interpret(String line, InterpreterContext interpreterContext) {
try {
LOGGER.info("***********sessionMap size:" + userSessionMap.size());
for (Map.Entry entry : userSessionMap.entrySet()) {
LOGGER.info("Session {}, user:{}", entry.getValue(), entry.getKey());
}
if (userSessionMap.get(interpreterContext.getAuthenticationInfo().getUser()) == null) {
try {
userSessionMap.put(
Expand Down Expand Up @@ -146,7 +150,7 @@ public Scheduler getScheduler() {
if (concurrentSQL()) {
int maxConcurrency = 10;
return SchedulerFactory.singleton().createOrGetParallelScheduler(
LivySparkInterpreter.class.getName() + this.hashCode(), maxConcurrency);
LivySparkSQLInterpreter.class.getName() + this.hashCode(), maxConcurrency);
} else {
Interpreter intp =
getInterpreterInTheSameSessionByClassName(LivySparkInterpreter.class.getName());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,9 +35,9 @@
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;

public class LivyIntegrationTest {
public class LivyInterpreterIT {

private static Logger LOGGER = LoggerFactory.getLogger(LivyIntegrationTest.class);
private static Logger LOGGER = LoggerFactory.getLogger(LivyInterpreterIT.class);
private static Cluster cluster;
private static Properties properties;

Expand Down Expand Up @@ -210,6 +210,56 @@ public void testSparkRInterpreter() {
// TODO (zjffdu), Livy's SparkRIntepreter has some issue, do it after livy-0.3 release.
}

@Test
public void testScheduler() throws InterruptedException {
if (!checkPreCondition()) {
return;
}

final LivySparkInterpreter sparkInterpreter = new LivySparkInterpreter(properties);
MyInterpreterOutputListener outputListener = new MyInterpreterOutputListener();
InterpreterOutput output = new InterpreterOutput(outputListener);
AuthenticationInfo authInfo1 = new AuthenticationInfo("user1");
final InterpreterContext context1 = new InterpreterContext("noteId", "paragraphId", "title",
"text", authInfo1, null, null, null, null, null, output);
AuthenticationInfo authInfo2 = new AuthenticationInfo("user2");
final InterpreterContext context2 = new InterpreterContext("noteId", "paragraphId", "title",
"text", authInfo2, null, null, null, null, null, output);
sparkInterpreter.open();
// initialize session first
sparkInterpreter.interpret("sc.version", context1);
sparkInterpreter.interpret("sc.version", context2);

long startTime = System.currentTimeMillis();
Thread thread1 = new Thread() {
@Override
public void run() {
InterpreterResult result = sparkInterpreter.interpret("Thread.sleep(1000*10)", context1);
assertEquals(InterpreterResult.Code.SUCCESS, result.code());
assertEquals(InterpreterResult.Type.TEXT, result.type());
}
};
Thread thread2 = new Thread() {
@Override
public void run() {
InterpreterResult result = sparkInterpreter.interpret("Thread.sleep(1000*10)", context2);
assertEquals(InterpreterResult.Code.SUCCESS, result.code());
assertEquals(InterpreterResult.Type.TEXT, result.type());
}
};
thread1.start();
thread2.start();
thread1.join();
thread2.join();
long endTime = System.currentTimeMillis();
long timeCost = endTime - startTime;
assertTrue("jobs in 2 sessions are not scheduled parallelly", timeCost < 20*1000);


// close sessions
sparkInterpreter.close();
}

public static class MyInterpreterOutputListener implements InterpreterOutputListener {
private StringBuilder outputAppended = new StringBuilder();
private StringBuilder outputUpdated = new StringBuilder();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -296,7 +296,7 @@ public void close() {
@Override
public InterpreterResult interpret(String st, InterpreterContext context) {
if (logger.isDebugEnabled()) {
logger.debug("st:\n{}", st);
logger.debug("st:\n{},\nuser:{}", st, context.getAuthenticationInfo().getUser());
}

FormType form = getFormType();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -427,6 +427,11 @@ public void onPostExecute(String script) {
hookListener.onPostExecute(script);
}

@Override
public String getUser() {
return context.getAuthenticationInfo().getUser();
}

@Override
protected Object jobRun() throws Throwable {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ThreadFactory;

/**
*
Expand Down Expand Up @@ -53,10 +54,18 @@ public ExecutorService createOrGet(String name) {
return createOrGet(name, 100);
}

public ExecutorService createOrGet(String name, int numThread) {
public ExecutorService createOrGet(final String name, int numThread) {
synchronized (executor) {
if (!executor.containsKey(name)) {
executor.put(name, Executors.newScheduledThreadPool(numThread));
executor.put(name, Executors.newScheduledThreadPool(numThread, new ThreadFactory(){
int i = 0;
@Override
public Thread newThread(Runnable r) {
Thread thread = new Thread(r);
thread.setName(name + "-" + i++);
return thread;
}
}));
}
return executor.get(name);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,118 @@
/*
* 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.zeppelin.scheduler;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.*;
import java.util.concurrent.ExecutorService;

/**
* Each user use one FIFOScheduler
*/
public class FIFOPerUserScheduler implements Scheduler {

private static Logger LOGGER = LoggerFactory.getLogger(FIFOScheduler.class);

private ExecutorService executor;
private SchedulerListener listener;
boolean terminate = false;
private String name;
private int maxUser = 10;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like unused var. Is it?


private Map<String, FIFOScheduler> schedulerMap = new HashMap<>();

public FIFOPerUserScheduler(String name, ExecutorService executor, SchedulerListener listener) {
this.name = name;
this.executor = executor;
this.listener = listener;
}

@Override
public String getName() {
return this.name;
}

@Override
public Collection<Job> getJobsWaiting() {
synchronized (schedulerMap) {
List<Job> waitingJobs = new ArrayList();
for (FIFOScheduler scheduler : schedulerMap.values()) {
waitingJobs.addAll(scheduler.getJobsWaiting());
}
return waitingJobs;
}
}

@Override
public Collection<Job> getJobsRunning() {
// return new ArrayList<>();
synchronized (schedulerMap) {
List<Job> runningJobs = new ArrayList();
for (FIFOScheduler scheduler : schedulerMap.values()) {
runningJobs.addAll(scheduler.getJobsRunning());
}
return runningJobs;
}
}

@Override
public void submit(Job job) {
synchronized (schedulerMap) {
FIFOScheduler scheduler = schedulerMap.get(job.getUser());
if (scheduler == null) {
scheduler = (FIFOScheduler) SchedulerFactory.singleton()
.createOrGetFIFOScheduler(this.name + "_" + job.getUser());
executor.execute(scheduler);
}
LOGGER.debug("Submitting job for owner:" + job.getUser());
scheduler.submit(job);
schedulerMap.put(job.getUser(), scheduler);
}
}

@Override
public Job removeFromWaitingQueue(String jobId) {
synchronized (schedulerMap) {
for (FIFOScheduler scheduler : schedulerMap.values()) {
Job job = scheduler.removeFromWaitingQueue(jobId);
if (job != null) {
schedulerMap.notify();
return job;
}
}
return null;
}
}

@Override
public void stop() {
synchronized (schedulerMap) {
for (FIFOScheduler scheduler : schedulerMap.values()) {
scheduler.stop();
}
}
}

@Override
public void run() {
// Do nothing, all the work is delegated to FIFOScheduler in schedulerMap
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ public class FIFOScheduler implements Scheduler {
private ExecutorService executor;
private SchedulerListener listener;
boolean terminate = false;
Job runningJob = null;
volatile Job runningJob = null;
private String name;

static Logger LOGGER = LoggerFactory.getLogger(FIFOScheduler.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,21 +107,25 @@ public Job(String jobId, String jobName, JobListener listener) {
this(jobId, jobName, listener, JobProgressPoller.DEFAULT_INTERVAL_MSEC);
}

public Job(String jobId, String jobName, JobListener listener, long progressUpdateIntervalMs) {
public Job(String jobId, String jobName, JobListener listener,
long progressUpdateIntervalMs) {
this.jobName = jobName;
this.listener = listener;
this.progressUpdateIntervalMs = progressUpdateIntervalMs;

dateCreated = new Date();
id = jobId;

setStatus(Status.READY);
}

public String getId() {
return id;
}

public String getUser() {
return null;
}

@Override
public int hashCode() {
return id.hashCode();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ public void run() {
try {
queue.wait(500);
} catch (InterruptedException e) {
LOGGER.error("Exception in MockInterpreterAngular while interpret queue.wait", e);
LOGGER.error("Exception in ParallelScheduler while interpret queue.wait", e);
}
continue;
}
Expand Down
Loading