Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add distributor which schedule task to be fairly #333

Merged
merged 4 commits into from
Nov 23, 2018
Merged
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
10 changes: 5 additions & 5 deletions fe/src/main/java/org/apache/doris/catalog/Catalog.java
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,7 @@
import org.apache.doris.load.LoadErrorHub;
import org.apache.doris.load.LoadJob;
import org.apache.doris.load.LoadJob.JobState;
import org.apache.doris.load.routineload.RoutineLoad;
import org.apache.doris.load.routineload.RoutineLoadManager;
import org.apache.doris.master.Checkpoint;
import org.apache.doris.master.MetaHelper;
import org.apache.doris.metric.MetricRepo;
Expand Down Expand Up @@ -241,7 +241,7 @@ public class Catalog {
private ConcurrentHashMap<String, Cluster> nameToCluster;

private Load load;
private RoutineLoad routineLoad;
private RoutineLoadManager routineLoadManager;
private ExportMgr exportMgr;
private Clone clone;
private Alter alter;
Expand Down Expand Up @@ -372,7 +372,7 @@ private Catalog() {
this.idToDb = new ConcurrentHashMap<>();
this.fullNameToDb = new ConcurrentHashMap<>();
this.load = new Load();
this.routineLoad = new RoutineLoad();
this.routineLoadManager = new RoutineLoadManager();
this.exportMgr = new ExportMgr();
this.clone = new Clone();
this.alter = new Alter();
Expand Down Expand Up @@ -4250,8 +4250,8 @@ public Load getLoadInstance() {
return this.load;
}

public RoutineLoad getRoutineLoadInstance() {
return routineLoad;
public RoutineLoadManager getRoutineLoadInstance() {
return routineLoadManager;
}

public ExportMgr getExportMgr() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,21 @@

package org.apache.doris.load.routineload;

public class KafkaRoutineLoadProgress {
import java.util.Map;

private String partitionName;
private long offset;
/**
* this is description of kafka routine load progress
* the data before offset was already loaded in doris
*/
public class KafkaProgress {

private Map<Integer, Long> partitionIdToOffset;

public Map<Integer, Long> getPartitionIdToOffset() {
return partitionIdToOffset;
}

public void setPartitionIdToOffset(Map<Integer, Long> partitionIdToOffset) {
this.partitionIdToOffset = partitionIdToOffset;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,13 @@
package org.apache.doris.load.routineload;

import com.google.common.base.Strings;
import com.google.gson.Gson;
import org.apache.doris.catalog.Catalog;
import org.apache.doris.catalog.Database;
import org.apache.doris.common.MetaNotFoundException;
import org.apache.doris.common.SystemIdGenerator;
import org.apache.doris.system.SystemInfoService;
import org.apache.doris.thrift.TResourceInfo;
import org.apache.doris.thrift.TTaskType;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.PartitionInfo;
import org.apache.logging.log4j.LogManager;
Expand All @@ -34,6 +34,7 @@
import java.util.ArrayList;
import java.util.List;
import java.util.Properties;
import java.util.UUID;

public class KafkaRoutineLoadJob extends RoutineLoadJob {
private static final Logger LOG = LogManager.getLogger(KafkaRoutineLoadJob.class);
Expand All @@ -43,13 +44,13 @@ public class KafkaRoutineLoadJob extends RoutineLoadJob {

private String serverAddress;
private String topic;
// optional
// optional, user want to load partitions.
private List<Integer> kafkaPartitions;

public KafkaRoutineLoadJob() {
}

public KafkaRoutineLoadJob(long id, String name, String userName, long dbId, long tableId,
public KafkaRoutineLoadJob(String id, String name, String userName, long dbId, long tableId,
String partitions, String columns, String where, String columnSeparator,
int desireTaskConcurrentNum, JobState state, DataSourceType dataSourceType,
int maxErrorNum, TResourceInfo resourceInfo, String serverAddress, String topic) {
Expand All @@ -59,21 +60,34 @@ public KafkaRoutineLoadJob(long id, String name, String userName, long dbId, lon
this.topic = topic;
}

public KafkaProgress getProgress() {
Gson gson = new Gson();
return gson.fromJson(this.progress, KafkaProgress.class);
}

@Override
public List<RoutineLoadTask> divideRoutineLoadJob(int currentConcurrentTaskNum) {
// divide kafkaPartitions into tasks
List<KafkaRoutineLoadTask> kafkaRoutineLoadTaskList = new ArrayList<>();
for (int i = 0; i < currentConcurrentTaskNum; i++) {
// TODO(ml): init load task
kafkaRoutineLoadTaskList.add(new KafkaRoutineLoadTask(getResourceInfo(), 0L, TTaskType.PUSH,
dbId, tableId, 0L, 0L, 0L, SystemIdGenerator.getNextId()));
}
for (int i = 0; i < kafkaPartitions.size(); i++) {
kafkaRoutineLoadTaskList.get(i % currentConcurrentTaskNum).addKafkaPartition(kafkaPartitions.get(i));
public void divideRoutineLoadJob(int currentConcurrentTaskNum) {
writeLock();
try {
if (state == JobState.NEED_SCHEDULER) {
// divide kafkaPartitions into tasks
for (int i = 0; i < currentConcurrentTaskNum; i++) {
KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo(UUID.randomUUID().toString(), id);
routineLoadTaskInfoList.add(kafkaTaskInfo);
needSchedulerTaskInfoList.add(kafkaTaskInfo);
}
for (int i = 0; i < kafkaPartitions.size(); i++) {
((KafkaTaskInfo) routineLoadTaskInfoList.get(i % currentConcurrentTaskNum))
.addKafkaPartition(kafkaPartitions.get(i));
}
// change job state to running
state = JobState.RUNNING;
} else {
LOG.debug("Ignore to divide routine load job while job state {}", state);
}
} finally {
writeUnlock();
}
List<RoutineLoadTask> result = new ArrayList<>();
result.addAll(kafkaRoutineLoadTaskList);
return result;
}

@Override
Expand All @@ -99,6 +113,15 @@ public int calculateCurrentConcurrentTaskNum() throws MetaNotFoundException {
return Math.min(partitionNum, Math.min(desireTaskConcurrentNum, aliveBeNum));
}

@Override
public RoutineLoadTask createTask(RoutineLoadTaskInfo routineLoadTaskInfo, long beId) {
return new KafkaRoutineLoadTask(getResourceInfo(),
beId, getDbId(), getTableId(),
0L, 0L, 0L, getColumns(), getWhere(), getColumnSeparator(),
(KafkaTaskInfo) routineLoadTaskInfo,
getProgress());
}

private void updatePartitions() {
// fetch all of kafkaPartitions in topic
if (kafkaPartitions == null || kafkaPartitions.size() == 0) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,27 +17,29 @@

package org.apache.doris.load.routineload;

import com.google.common.collect.Lists;
import org.apache.doris.thrift.TResourceInfo;
import org.apache.doris.thrift.TTaskType;

import java.util.List;
import java.util.HashMap;
import java.util.Map;

public class KafkaRoutineLoadTask extends RoutineLoadTask {

private List<Integer> kafkaPartitions;

public KafkaRoutineLoadTask(TResourceInfo resourceInfo, long backendId, TTaskType taskType,
long dbId, long tableId, long partitionId, long indexId, long tabletId, long signature) {
super(resourceInfo, backendId, taskType, dbId, tableId, partitionId, indexId, tabletId, signature);
this.kafkaPartitions = Lists.newArrayList();
}
public class KafkaRoutineLoadTask extends RoutineLoadTask {

public void addKafkaPartition(int partition) {
kafkaPartitions.add(partition);
private Map<Integer, Long> partitionIdToOffset;

public KafkaRoutineLoadTask(TResourceInfo resourceInfo, long backendId,
long dbId, long tableId, long partitionId, long indexId, long tabletId,
String columns, String where, String columnSeparator,
KafkaTaskInfo kafkaTaskInfo, KafkaProgress kafkaProgress) {
super(resourceInfo, backendId, TTaskType.STREAM_LOAD, dbId, tableId, partitionId, indexId, tabletId,
kafkaTaskInfo.getId(), columns, where, columnSeparator, RoutineLoadJob.DataSourceType.KAFKA);
this.partitionIdToOffset = new HashMap<>();
kafkaTaskInfo.getPartitions().parallelStream().forEach(entity ->
partitionIdToOffset.put(entity, kafkaProgress.getPartitionIdToOffset().get(entity)));
}

public List<Integer> getKafkaPartitions() {
return kafkaPartitions;
public Map<Integer, Long> getPartitionIdToOffset() {
return partitionIdToOffset;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
// 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.doris.load.routineload;

import org.apache.doris.common.SystemIdGenerator;

import java.util.ArrayList;
import java.util.List;
import java.util.UUID;

public class KafkaTaskInfo extends RoutineLoadTaskInfo {

private List<Integer> partitions;

public KafkaTaskInfo(String id, String jobId) {
super(id, jobId);
this.partitions = new ArrayList<>();
}

public KafkaTaskInfo(KafkaTaskInfo kafkaTaskInfo) {
super(UUID.randomUUID().toString(), kafkaTaskInfo.getJobId());
this.partitions = kafkaTaskInfo.getPartitions();
}

public void addKafkaPartition(int partition) {
partitions.add(partition);
}

public List<Integer> getPartitions() {
return partitions;
}


}
Loading