Skip to content

Commit 1c76e95

Browse files
improve scalability of sql queries
1 parent 0f20889 commit 1c76e95

9 files changed

Lines changed: 309 additions & 44 deletions

File tree

Lines changed: 113 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,113 @@
1+
/**
2+
The MIT License (MIT)
3+
4+
Copyright (c) 2010-2019 head systems, ltd
5+
6+
Permission is hereby granted, free of charge, to any person obtaining a copy of
7+
this software and associated documentation files (the "Software"), to deal in
8+
the Software without restriction, including without limitation the rights to
9+
use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
10+
the Software, and to permit persons to whom the Software is furnished to do so,
11+
subject to the following conditions:
12+
13+
The above copyright notice and this permission notice shall be included in all
14+
copies or substantial portions of the Software.
15+
16+
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
17+
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
18+
FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
19+
COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
20+
IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
21+
CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
22+
23+
*/
24+
25+
package su.interference.sql;
26+
27+
import su.interference.core.Config;
28+
29+
import java.io.Serializable;
30+
import java.util.List;
31+
import java.util.Map;
32+
import java.util.concurrent.Callable;
33+
34+
/**
35+
* @author Yuriy Glotanov
36+
* @since 1.0
37+
*/
38+
39+
public class FrameApiJoin implements Serializable, Callable<FrameApiJoin> {
40+
private int nodeId;
41+
private final transient SQLCursor cur;
42+
private final transient FrameApi bd1;
43+
private final transient FrameApi bd2;
44+
private final transient FrameJoinTask frameJoinTask;
45+
private final long leftAllocId;
46+
private final long rightAllocId;
47+
private List<Object> result;
48+
49+
public FrameApiJoin(int nodeId, SQLCursor cur, FrameApi bd1, FrameApi bd2, Map<Integer, Map<String, FrameApiJoin>> joins) {
50+
this.nodeId = nodeId;
51+
this.cur = cur;
52+
this.bd1 = bd1;
53+
this.bd2 = bd2;
54+
this.leftAllocId = bd1.getAllocId();
55+
this.rightAllocId = bd2 == null ? 0 : bd2.getAllocId();
56+
if (nodeId == Config.getConfig().LOCAL_NODE_ID) {
57+
frameJoinTask = cur.buildFrameJoinTask(nodeId, bd1, bd2);
58+
} else {
59+
frameJoinTask = null;
60+
joins.get(nodeId).put(this.getKey(), this);
61+
}
62+
}
63+
64+
public FrameApiJoin call() throws Exception {
65+
if (nodeId == Config.getConfig().LOCAL_NODE_ID) {
66+
final FrameJoinTask frameJoinTask = cur.buildFrameJoinTask(nodeId, bd1, bd2);
67+
result = frameJoinTask.call();
68+
}
69+
return this;
70+
}
71+
72+
protected String getKey() {
73+
return this.leftAllocId + "-" + this.rightAllocId;
74+
}
75+
76+
@Override
77+
public boolean equals(Object obj) {
78+
return this.getKey().equals(((FrameApiJoin)obj).getKey());
79+
}
80+
81+
@Override
82+
public int hashCode() {
83+
return this.getKey().hashCode();
84+
}
85+
86+
public int getNodeId() {
87+
return nodeId;
88+
}
89+
90+
public void setNodeId(int nodeId) {
91+
this.nodeId = nodeId;
92+
}
93+
94+
public FrameJoinTask getFrameJoinTask() {
95+
return frameJoinTask;
96+
}
97+
98+
public long getLeftAllocId() {
99+
return leftAllocId;
100+
}
101+
102+
public long getRightAllocId() {
103+
return rightAllocId;
104+
}
105+
106+
public List<Object> getResult() {
107+
return result;
108+
}
109+
110+
public void setResult(List<Object> result) {
111+
this.result = result;
112+
}
113+
}

src/main/java/su/interference/sql/FrameJoinTask.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -101,6 +101,7 @@ public List<Object> call() throws Exception {
101101
ResultSetEntity rsa = (ResultSetEntity)c1.getAnnotation(ResultSetEntity.class);
102102
final boolean c1rs = rsa!=null?true:false;
103103

104+
/*
104105
if (cur.getType() == Cursor.MASTER_TYPE && nodeId != Config.getConfig().LOCAL_NODE_ID && this.leftfs) {
105106
Metrics.get("remoteTask").start();
106107
final TransportEvent transportEvent = new SQLEvent(nodeId, cur.getCursorId(), bd1.getAllocId(), bd2==null?0:bd2.getAllocId(), bd2==null?null:bd2.getClass().getSimpleName(), 0, null, null, 0, true);
@@ -114,6 +115,7 @@ public List<Object> call() throws Exception {
114115
return rs;
115116
}
116117
}
118+
*/
117119

118120
Metrics.get("localTask").start();
119121
final ArrayList<Object> drs1 = bd1.getFrameEntities(s);
Lines changed: 79 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,79 @@
1+
/**
2+
The MIT License (MIT)
3+
4+
Copyright (c) 2010-2019 head systems, ltd
5+
6+
Permission is hereby granted, free of charge, to any person obtaining a copy of
7+
this software and associated documentation files (the "Software"), to deal in
8+
the Software without restriction, including without limitation the rights to
9+
use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
10+
the Software, and to permit persons to whom the Software is furnished to do so,
11+
subject to the following conditions:
12+
13+
The above copyright notice and this permission notice shall be included in all
14+
copies or substantial portions of the Software.
15+
16+
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
17+
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
18+
FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
19+
COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
20+
IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
21+
CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
22+
23+
*/
24+
25+
package su.interference.sql;
26+
27+
import su.interference.core.Config;
28+
import su.interference.exception.InternalException;
29+
import su.interference.metrics.Metrics;
30+
import su.interference.persistent.Cursor;
31+
import su.interference.transport.SQLEvent;
32+
import su.interference.transport.TransportContext;
33+
import su.interference.transport.TransportEvent;
34+
35+
import java.util.List;
36+
import java.util.Map;
37+
import java.util.concurrent.Callable;
38+
39+
/**
40+
* @author Yuriy Glotanov
41+
* @since 1.0
42+
*/
43+
44+
public class RemoteTask implements Callable<Boolean> {
45+
private final Cursor cur;
46+
private final int nodeId;
47+
private final Map<String, FrameApiJoin> joins;
48+
private final String rightType;
49+
50+
public RemoteTask(Cursor cur, int nodeId, Map<String, FrameApiJoin> joins, String rightType) {
51+
this.cur = cur;
52+
this.nodeId = nodeId;
53+
this.joins = joins;
54+
this.rightType = rightType;
55+
}
56+
57+
public Boolean call() throws Exception {
58+
Metrics.get("remoteTask").start();
59+
final TransportEvent transportEvent = new SQLEvent(nodeId, cur.getCursorId(), joins, rightType, 0, null, null, 0, true);
60+
TransportContext.getInstance().send(transportEvent);
61+
transportEvent.getLatch().await();
62+
Metrics.get("remoteTask").stop();
63+
if (!transportEvent.isFail()) {
64+
final List<FrameApiJoin> rs = ((SQLEvent) transportEvent).getCallback().getResult().getResultSet();
65+
if (rs.size() != joins.size()) {
66+
throw new InternalException();
67+
}
68+
Metrics.get("recordRCount").put(rs.size());
69+
for (FrameApiJoin j : rs) {
70+
joins.get(j.getKey()).setResult(j.getResult());
71+
}
72+
} else {
73+
for (Map.Entry<String, FrameApiJoin> entry : joins.entrySet()) {
74+
entry.getValue().setNodeId(Config.getConfig().LOCAL_NODE_ID);
75+
}
76+
}
77+
return true;
78+
}
79+
}

src/main/java/su/interference/sql/SQLCursor.java

Lines changed: 56 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -50,12 +50,14 @@ public class SQLCursor implements FrameIterator {
5050

5151
private final int id;
5252
private ResultSet target;
53-
private boolean done;
5453
private ExecutorService exec = SQLJoinThreadPool.getThreadPool();
55-
private ArrayList<FrameJoinTask> tasks;
54+
private List<FrameApiJoin> tasks;
55+
private List<FrameApiJoin> tasks_;
56+
private Map<Integer, Map<String, FrameApiJoin>> joins;
5657
private FrameData bdnext;
5758
private FrameHolder current;
5859
private int ptr = 0;
60+
private boolean sent;
5961
private Cursor cur;
6062
private Session s;
6163
private ArrayList<SQLColumn> rscols;
@@ -66,6 +68,7 @@ public class SQLCursor implements FrameIterator {
6668
private final FrameIterator lbi;
6769
private final FrameIterator rbi;
6870
private final List<Integer> objectIds;
71+
private String rightType;
6972
private boolean leftFS;
7073
private final boolean furtherUseUC;
7174
private final SQLColumn joinedCC;
@@ -104,7 +107,9 @@ public SQLCursor (int id, FrameIterator lbi, FrameIterator rbi, NestedCondition
104107
}
105108
current = new FrameHolder(target);
106109

107-
tasks = new ArrayList<FrameJoinTask>();
110+
tasks = new ArrayList<FrameApiJoin>();
111+
tasks_ = new ArrayList<FrameApiJoin>();
112+
joins = new HashMap<>();
108113

109114
//rebuild column set for sqlcursor iterator
110115
final SQLCursor cursor_ = lbi.getType() == FrameIterator.TYPE_CURSOR ? (SQLCursor) lbi : rbi != null && rbi.getType() == FrameIterator.TYPE_CURSOR ? (SQLCursor) rbi : null;
@@ -173,29 +178,48 @@ public int getObjectId() {
173178
return target.getObjectId();
174179
}
175180

181+
protected FrameJoinTask buildFrameJoinTask(int nodeId, FrameApi bd1, FrameApi bd2) {
182+
return new FrameJoinTask(cur, bd1, bd2, target, rscols, nc, id, nodeId, last, lbi.isLeftfs(), hmap, s);
183+
}
184+
176185
public void build() throws InternalException, IOException, ClassNotFoundException, InvocationTargetException, NoSuchMethodException, InstantiationException, IllegalAccessException {
177186
final Integer[] ns = TransportContext.getInstance().getOnlineNodesWithLocal();
187+
for (Integer nodeId : ns) {
188+
if (nodeId != Config.getConfig().LOCAL_NODE_ID) {
189+
joins.put(nodeId, new HashMap<>());
190+
}
191+
}
178192
int i = 0;
179193
//int tnode = 0;
180194

181195
while (lbi.hasNextFrame()) {
182-
FrameApi bd1 = lbi.nextFrame();
196+
final FrameApi bd1 = lbi.nextFrame();
183197

184198
if (rbi == null) {
185-
tasks.add(new FrameJoinTask(cur, bd1, null, target, rscols, nc, id, ns[i], last, lbi.isLeftfs(), null, s));
199+
tasks.add(new FrameApiJoin(ns[i], this, bd1, null, joins));
186200
i++;
187201
if (i == ns.length) { i = 0; }
188202
} else {
189203
while (rbi.hasNextFrame()) {
190-
FrameApi bd2 = rbi.nextFrame();
191-
tasks.add(new FrameJoinTask(cur, bd1, bd2, target, rscols, nc, id, ns[i], last, lbi.isLeftfs(), hmap, s));
204+
final FrameApi bd2 = rbi.nextFrame();
205+
if (rightType == null) {
206+
rightType = bd2.getClass().getSimpleName();
207+
}
208+
tasks.add(new FrameApiJoin(ns[i], this, bd1, bd2, joins));
192209
i++;
193210
if (i == ns.length) { i = 0; }
194211
}
195212
rbi.resetIterator();
196213
}
197214
}
198-
logger.debug("SQL cursor is build: tasks amount = "+tasks.size()+", use NC check = "+last);
215+
logger.debug("SQL cursor is build: local tasks amount = "+tasks.size()+", use NC check = "+last);
216+
if (!sent) {
217+
for (Map.Entry<Integer, Map<String, FrameApiJoin>> entry : joins.entrySet()) {
218+
final RemoteTask rt = new RemoteTask(cur, entry.getKey(), entry.getValue(), rightType);
219+
exec.submit(rt);
220+
}
221+
sent = true;
222+
}
199223
}
200224

201225
public void stream() throws Exception {
@@ -241,27 +265,38 @@ public synchronized FrameData nextFrame() {
241265
}
242266

243267
private synchronized FrameData nextFrame2() throws InternalException {
244-
boolean done = !(ptr<tasks.size());
268+
boolean done = !(ptr < tasks.size());
245269
FrameData ret = current.getFrame(done);
246270

247-
while (!done&&ret==null) {
271+
while (!done && ret==null) {
272+
final ArrayList<Future<FrameApiJoin>> flist = new ArrayList<Future<FrameApiJoin>>();
248273
try {
249-
ArrayList<Future<List<Object>>> flist = new ArrayList<Future<List<Object>>>();
250-
for (int i=0; i<BATCH_SIZE; i++) {
251-
if ((ptr + i)<tasks.size()) {
252-
FrameJoinTask jt = tasks.get(ptr + i);
253-
flist.add(exec.submit(jt));
274+
for (int i=0; i < BATCH_SIZE; i++) {
275+
if ((ptr + i) < tasks.size()) {
276+
final FrameApiJoin j = tasks.get(ptr + i);
277+
flist.add(exec.submit(j));
254278
}
255279
}
256-
for (Future<List<Object>> f : flist) {
257-
List<Object> ol = f.get();
258-
logger.debug("SQL cursor next frame: the jointask call returned " + ol.size() + " records");
259-
for (Object o : ol) {
260-
target.persist(o, s);
280+
for (Future<FrameApiJoin> f : flist) {
281+
final FrameApiJoin j = f.get();
282+
final List<Object> ol = j.getResult();
283+
if (ol == null) {
284+
tasks_.add(j);
285+
} else {
286+
logger.debug("SQL cursor next frame: the jointask call returned " + ol.size() + " records");
287+
for (Object o : ol) {
288+
target.persist(o, s);
289+
}
261290
}
262291
}
263292
ptr = ptr + BATCH_SIZE;
264-
done = !(ptr<tasks.size());
293+
done = !(ptr < tasks.size());
294+
if (done && tasks_.size() > 0) {
295+
done = false;
296+
tasks = tasks_;
297+
tasks_ = new ArrayList<>();
298+
ptr = 0;
299+
}
265300
ret = current.getFrame(done);
266301
} catch (Exception e) {
267302
e.printStackTrace();

src/main/java/su/interference/sql/SQLJoinThreadPool.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@ this software and associated documentation files (the "Software"), to deal in
3434

3535
public class SQLJoinThreadPool {
3636

37-
private static ExecutorService exec = Executors.newFixedThreadPool(4);
37+
private static ExecutorService exec = Executors.newFixedThreadPool(8);
3838

3939
public static ExecutorService getThreadPool() {
4040
return exec;

src/main/java/su/interference/sql/SQLSelect.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -328,7 +328,7 @@ private final void parseSQL (String s, Cursor cur, Session sn) throws Exception
328328

329329
final Integer[] ns = TransportContext.getInstance().getOnlineNodes();
330330
for (Integer nodeId : TransportContext.getInstance().getOnlineNodes()) {
331-
final TransportEvent transportEvent = new SQLEvent(nodeId, cursor.getCursorId(), 0, 0, null,
331+
final TransportEvent transportEvent = new SQLEvent(nodeId, cursor.getCursorId(), null, null,
332332
this.join.getTargetId(), cursor.getSql(), this.join.getResultTargetName(), sn.getTransaction().getTransId(), false);
333333
TransportContext.getInstance().send(transportEvent);
334334
logger.debug("create remote cursor on node " + nodeId);

0 commit comments

Comments
 (0)