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

[TUBEMQ-149]Some of the consumers stop consuming their corresponding partitions and never release the partition to others[addendum] #96

Merged
merged 1 commit into from
May 25, 2020
Merged
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 @@ -340,11 +340,7 @@ protected void errReqRelease(String partitionKey, long usedToken, boolean isLast
if (oldUsedToken != null && oldUsedToken == usedToken) {
partitionUsedMap.remove(partitionKey);
partitionExt.setLastPackConsumed(isLastPackConsumed);
try {
indexPartition.offer(partitionKey);
} catch (Throwable e) {
//
}
releaseIdlePartition(-1, partitionKey);
}
}
}
Expand All @@ -365,16 +361,7 @@ protected void succRspRelease(String partitionKey, String topicName,
partitionExt.setLastPackConsumed(isLastPackConsumed);
long waitDlt =
partitionExt.procConsumeResult(isFilterConsume);
if (waitDlt > 10) {
timeouts.put(partitionKey,
timer.newTimeout(new TimeoutTask(partitionKey), waitDlt, TimeUnit.MILLISECONDS));
} else {
try {
indexPartition.offer(partitionKey);
} catch (Throwable e) {
//
}
}
releaseIdlePartition(waitDlt, partitionKey);
}
}
}
Expand All @@ -398,21 +385,26 @@ public void errRspRelease(String partitionKey, String topicName,
long waitDlt =
partitionExt.procConsumeResult(isFilterConsume, reqProcType,
errCode, msgSize, isEscLimit, limitDlt, curDataDlt, false);
if (waitDlt > 10) {
timeouts.put(partitionKey,
timer.newTimeout(new TimeoutTask(partitionKey), waitDlt, TimeUnit.MILLISECONDS));
} else {
try {
indexPartition.offer(partitionKey);
} catch (Throwable e) {
//
}
}
releaseIdlePartition(waitDlt, partitionKey);
}
}
}
}

private void releaseIdlePartition(long waitDlt, String partitionKey) {
if (waitDlt > 10) {
TimeoutTask timeoutTask = new TimeoutTask(partitionKey);
timeouts.put(partitionKey,
timer.newTimeout(timeoutTask, waitDlt, TimeUnit.MILLISECONDS));
} else {
try {
indexPartition.offer(partitionKey);
} catch (Throwable e) {
//
}
}
}

/**
* Close the remote data cache
*/
Expand All @@ -427,11 +419,7 @@ public void close() {
timer = null;
}
for (int i = this.waitCont.get() + 1; i > 0; i--) {
try {
indexPartition.offer("------");
} catch (Throwable e) {
//
}
releaseIdlePartition(-1, "------");
}
}
}
Expand Down Expand Up @@ -663,6 +651,31 @@ public void resumeTimeoutConsumePartitions(long allowedPeriodTimes) {
}
}
}
// add timeout expired check
if (!timeouts.isEmpty()) {
List<String> partKeys = new ArrayList<String>();
partKeys.addAll(timeouts.keySet());
Timeout timeout1 = null;
for (String keyId : partKeys) {
timeout1 = timeouts.get(keyId);
if (timeout1 != null && timeout1.isExpired()) {
timeout1 = timeouts.remove(keyId);
if (timeout1 != null) {
PartitionExt partitionExt = partitionMap.get(keyId);
if (partitionExt != null) {
if (!indexPartition.contains(keyId)) {
try {
indexPartition.offer(keyId);
} catch (Throwable e) {
//
}
}
}
}
}
}

}
}

private void waitPartitions(List<String> partitionKeys, long inUseWaitPeriodMs) {
Expand Down Expand Up @@ -787,9 +800,15 @@ private boolean hasPartitionWait() {
public class TimeoutTask implements TimerTask {

private String indexId;
private long createTime = 0L;

public TimeoutTask(final String indexId) {
this.indexId = indexId;
this.createTime = System.currentTimeMillis();
}

public long getCreateTime() {
return this.createTime;
}

@Override
Expand Down