Commit Graph

112 Commits

Author SHA1 Message Date
93a3577baa Support multi partition column when creating table (#1574)
When creating table with OLAP engine, use can specify multi parition columns.
eg:

PARTITION BY RANGE(`date`, `id`)
(
    PARTITION `p201701_1000` VALUES LESS THAN ("2017-02-01", "1000"),
    PARTITION `p201702_2000` VALUES LESS THAN ("2017-03-01", "2000"),
    PARTITION `p201703_all`  VALUES LESS THAN ("2017-04-01")
)

Notice that load by hadoop cluster does not support multi parition column table.
2019-08-05 16:16:43 +08:00
938c6d4cdf Thrown TabletQuorumFailedException in commitTxn (#1575)
The TabletQuorumFailedException will be thrown in commitTxn while the success replica num of tablet is less then quorom replica num.
The Hadoop load does not handle this exception because the push task will retry it later.
The streaming broker, insert, stream and mini load will catch this exception and abort the txn after that.
2019-08-04 15:54:03 +08:00
0694b6a6fa Fix bugs of Broker load (#1546)
Use same UUID as query ID and load ID of a load execution plan.
Each load execution plan has a load ID, and as a plan, there is also a query ID.
We can use same UUID as query ID and load ID, for tracing the load process more easily.

Change the load ID when retrying a load execution plan.
When a load execution plan retry, the load ID should be changed, otherwise BE can not
distinguish the old and new load requests.

Cancel the running loading task when cancelling the broker load.
When user cancel a broker load, the running loading task should also be cancelled, or
it may occupies the worker thread for a long time.

Remove the unnecessary query report when doing load execution plan.
Only the last query report is needed.

Add a new BE config tablet_writer_rpc_timeout_sec.
It is used for RPC of tablet sink. The default is 600 seconds. which is long enough for flushing
about 6GB data. The long timeout config will reduce the possibility of encountering fail to send batch error when loading.

Use streaming_load_max_mb instead of mini_load_max_mb in BE config.

Add more logs for tracing a broker load process easily.
2019-07-27 20:17:05 +08:00
69040572fb Use different ID instead of table ID for base index of an OLAP table (#1524) 2019-07-23 15:48:45 +08:00
1f3f3f76a2 Fix the duplicated request bug of mini load (#1504)
The function of miniLoadBegin will return the txn_id.
If the backend sends the duplicated request to frontend, frontend will return the txn_id which was created by the same mini load.

The issue is that frontend returns the txn_id when the last same request hasn't been begun the txn.
The frontend returns the zero which is initialized txn_id and the be could not execute the load plan with a error txn_id.

The commit conbines the `createLoadJob` and `execute` together in the write lock. It protects the atomicity of `create` and `beginTxn`.
So the duplicated request cannot get the txn id before the last same request is finished.
2019-07-18 23:52:12 +08:00
2551248a52 Support grant GRANT_PRIV on database or table level (#1472)
Currently, GRANT_PRIV can only be granted on global level, which means
it can only be granted on *.*. Grant it on db.* or db.tbl are not allowed.

This will not be able to meet the requirement to create a user who has privilege
to grant privileges to other users on specified database or table, such as:

GRANT SELECT_PRIV ON db1.* TO cmy@'%';

So I extend the range of GRANT_PRIV. User can now grant GRANT_PRIV on
database or even table level, such as:

GRANT GRANT_PRIV ON db1.* TO cmy@'%';

And after being granted, the user cmy@'%' can now grant GRANT_PRIV on db1.* to
other users.
2019-07-16 19:25:18 +08:00
0d48a3961c Refactor Storage Engine (#1478)
NOTE: This patch would modify all Backend's data.
And this will cause a very long time to restart be.
So if you want to interferer your product environment,
you should upgrade backend one by one.

1. Refactoring be is to clarify the structure the codes.
2. Use unique id to indicate a rowset.
   Nameing rowset with tablet_id and version will lead to
   many conflicts among compaction, clone, restore.
3. Extract an rowset interface to encapsulate rowsets
   with different format.
2019-07-15 21:18:22 +08:00
863eb83cb1 Delete deprecated code in Frontend (#1463)
1. Delete Clone/CloneJob/CloneChecker
    The old clone framework is deprecated, using TabletChecker/TabletScheduler instead
2. Delete old BackupJob/RestoreJob
3. Delete OP_DROP_USER edit log
4. Delete CLONE_DONE edit log
2019-07-12 13:34:05 +08:00
81f062dd4c Bug-fix: query es table would fail when thrift_port configuration not set (#1455) 2019-07-11 12:29:18 +08:00
9c96a688c3 Fix bug that user can set null default value to non-nullable column in create table stmt (#1453)
In create table stmt, column definition `k1 INT NOT NULL DEFAULT NULL`
should not be allowed
2019-07-10 23:48:29 +08:00
645f0a5279 Persist auth info in LoadJob (#1443)
The new class named 'AuthorizationInfo' is used to save the auth info in jobs.
The job doesn't need to retrieve the auth info by meta id which maybe throw the exception when db or table has been dropped or renamed.
The persistence of 'AuthorizationInfo' take effect in META_VERSION 56
2019-07-09 20:50:55 +08:00
bde362c3cd Modify insert operation's behavior (#1444)
Before changing default insert operation to streaming load, if the select result
of a insert stmt is empty, a label will still be returned to the user, and user
can use this label to check the insert load job's status.

After changing the insert operation, if the select result is empty, a exception
will be thrown to user client directly without any label.

This new usage pattern is not friendly to already existed users, which is forcing
them to change their way of using insert operation.

So I add a new FE config 'using_old_load_usage_pattern', default is false.
If set to true, a label will be returned to user even if the select result is empty.
2019-07-09 10:17:09 +08:00
7eab12a40e Support reading Parquet file when loading data (#1173) 2019-07-01 18:39:27 +08:00
6b83440b59 Get table name from DataSourceInfo instead of DataDesc (#1405) 2019-06-29 11:20:12 +08:00
5c1b4f641e Add report version for publish task (#1401) 2019-06-28 20:15:08 +08:00
e807064a88 Modify colocation creation logic (#1289) 2019-06-25 21:20:18 +08:00
322de9cd8e Add sql-function doc of cast_to_bigint (#1370) 2019-06-24 19:40:57 +08:00
120e7e9119 Add more UT for FEFunctions (#1344) 2019-06-21 21:54:14 +08:00
7550b2f09b Convert mini load to streaming mini load (#1323)
* This commit has brought contribution to streaming mini load
The operation of streaming mini load is sames as previous. Also, user can check the load by frontend.
The difference is that streaming mini load finish the task before reply of REST API while the non-streaming only register a load.

* When updating doris
Updating fe or be firstly are also supported. After fe and be are updated, the streaming mini load will take effect.

* For multi mini load
The non-streaming mini load still has been used by multi mini load. The behavior of multi mini load has not been changed.

* Add a interface named isSupportedFunction
This function is used to protect the correctness of new feature which consists of be and fe during updaing.
2019-06-21 19:34:50 +08:00
ea71277094 Support mysql client 8.0 connection fe (#1349)
for example:
mysql --default-auth=mysql_native_password -P9030 -utest -ptest123456 -hA.B.C.D
2019-06-21 19:15:34 +08:00
b002ba04d9 Fix the error of duplicated label (#1303) 2019-06-14 14:13:38 +08:00
ff0dd0d2da Support SSL authentication with Kafka in routine load job (#1235) 2019-06-07 16:29:01 +08:00
f424321625 Fix IllegalArgumentException in LoadManager (#1240) 2019-06-04 22:23:13 +08:00
309b779a7d Check colocate table name should be case-sensitive (#1224) 2019-05-30 22:47:22 +08:00
180d8e5cbd Modify some thirdparties (#1228)
1. Change Kafka java client from 2.0.0 to 0.10.1.1. Because high version client may not support low server server.
2. Enable SSL in librdkafka
2019-05-30 21:23:37 +08:00
fa4ac9f751 Replay GlobalVariable by Annotation (#1219) 2019-05-29 19:21:42 +08:00
f648bdd968 Fix datediff function (#1208) 2019-05-28 15:55:31 +08:00
f985ea99fc Add support column reference in LOAD statement (#1162) 2019-05-15 20:26:10 +08:00
ffe3eaa1a7 Implement adddate, days_add and from_unixtime function in FE (#1149) 2019-05-13 16:59:52 +08:00
15c9be4dfe Fix bug that balance task always choose high usage path (#1143) 2019-05-11 22:07:17 +08:00
ae18cebe0b Improve colocate table balance logic for backend added (#1139)
1. Improve colocate table balance logic for backend added
2. Add more comment
3. Break loop early
2019-05-11 21:49:51 +08:00
1eeb5ea891 Add str_to_date function in fe (#1118) 2019-05-09 17:20:44 +08:00
a08170fd50 Enhance the usabilities (#1100)
* Enhence the usabilities

1. Add metrics to monitor transactions and steaming load process in BE.
2. Modify BE config 'result_buffer_cancelled_interval_time' to 300s.
3. Modify FE config 'enable_metric_calculator' to true.
4. Add more log for tracing broker load process.
5. Modify the query report process, to cancel query immediately if some instance failed.

* Fix bugs
1. Avoid NullPointer when enabling colocation join with broker load
2. Return immediately when pull load task coordinator execution failed
2019-05-07 15:55:04 +08:00
588aa7bed3 Fix date_format function in fe (#1082) 2019-05-01 22:20:49 +08:00
1662d91877 Change the logic of RoutineLoadTaskScheduler (#1061)
1. TaskScheduler will process one task per round
2. TaskScheduler will be blocked till queue tasks a new task
3. TaskScheduler will submit tasks when queue is empty
4. Add a example of creating a broker table by BOS
5. Change syntax of show routine load job
2019-04-28 20:05:48 +08:00
60df7cdb8d fix ut bug (#1051) 2019-04-28 10:33:50 +08:00
0adb150da7 Fix ut bugs (#1046)
Also fix a metrics collection bug
2019-04-28 10:33:50 +08:00
4a95c53f07 Fix bug of listener (#1017)
* Fix bug of listener

* Change txnStateChangeListener to txnStateChangeCallback

* Fix the logic of beforeAborted
1. It task is not belong to job, the txn attachment will be set to null.
* Txn will be abort normally without attachment.
* Job will not be updated by this task which attachment is null.
2019-04-28 10:33:50 +08:00
3409ed41ac Reset commit offset if task aborted due to runtime error (#994) 2019-04-28 10:33:50 +08:00
1b5643c6fb Fix some bugs (#979)
1. Add Config.max_routine_load_concurrent_task_num instead of the old one
2. Fix a bug that SHOW ALTER TABLE COLUMN may throw Nullpointer exception
3. Fix some misspelling of docs
2019-04-28 10:33:50 +08:00
b7b66527ce Fix some load bugs (#961)
1. Use load job's timeout as its txn timeout
2. Add a new session variable 'forward_to_master' for SHOW PROC and ADMIN stmt
2019-04-28 10:33:50 +08:00
e352a08339 Change tips of show routine load task (#959)
1. Add pauseTimestamp
2. It will be set when job is paused and it will be removed when job is resumed
2019-04-28 10:33:50 +08:00
2b4d02b2fa Add error load log url for routine load job (#938) 2019-04-28 10:33:50 +08:00
8e0512e88d Move lock of routine load job (#934)
1. Moving lock of routine load job from inside of lock of txn to outside.
2. The process of routine load task commit or abort is following:
* lock job
      check task
  lock txn
      commit txn
  unlock txn
      commit task
* unlock job
3. The process of checking timeout txn will be ignored when there are related task of txn.
4. The relationship between task and txn will be removed when task timeout.
2019-04-28 10:33:50 +08:00
75674753c2 Add unit test for RoutineLoadManager and RoutineLoadJob (#881)
1. Add ut
2. Show history job when table has been deleted. Checking auth whatever tablename is null or not.
2019-04-28 10:33:50 +08:00
400d8a906f Optimize the consumer assignment of Kafka routine load job (#870)
1. Use a data consumer group to share a single stream load pipe with multi data consumers. This will increase the consuming speed of Kafka messages, as well as reducing the task number of routine
load job. 

Test results:

* 1 consumer, 1 partitions:
    consume time: 4.469s, rows: 990140, bytes: 128737139.  221557 rows/s, 28M/s
* 1 consumer, 3 partitions:
    consume time: 12.765s, rows: 2000143, bytes: 258631271. 156689 rows/s, 20M/s
    blocking get time(us): 12268241, blocking put time(us): 1886431
* 3 consumers, 3 partitions:
    consume time(all 3): 6.095s, rows: 2000503, bytes: 258631576. 328220 rows/s, 42M/s
    blocking get time(us): 1041639, blocking put time(us): 10356581

The next 2 cases show that we can achieve higher speed by adding more consumers. But the bottle neck transfers from Kafka consumer to Doris ingestion, so 3 consumers in a group is enough.

I also add a Backend config `max_consumer_num_per_group` to change the number of consumers in a data consumer group, and default value is 3.

In my test(1 Backend, 2 tablets, 1 replicas), 1 routine load task can achieve 10M/s, which is same as raw stream load.

2. Add OFFSET_BEGINNING and OFFSET_END support for Kafka routine load
2019-04-28 10:33:50 +08:00
cef2078cb8 Fix FE UT (#850) 2019-04-28 10:33:50 +08:00
e1c6ba8397 Add show proc of routine load and task (#818)
1. add show proc "/routine_loads" to show statistic of all of jobs and tasks
2. add show proc "/routine_loads/jobname" to show info of all of jobs named jobname
3. add show proc "/routine_loads/jobname/jobid" to show tasks belong to jobid
4. fix bug of allocateBeToTask
2019-04-28 10:33:50 +08:00
2e250482fd Modify routine load fe unit test (#803) 2019-04-28 10:33:50 +08:00
d213f922be Implement ShowRoutineLoadStmt and ShowRoutineLoadTaskStmt (#786)
1. ShowRoutineLoadStmt is sames like class description. It does not support show all of routine load job in all of db
2. ShowRoutineLoadTaskStmt is sames like class description. It does not support show all of routine laod task in all of job
3. Init partitionIdsToOffset in constructor of KafkaProgress
4. Change Create/Pause/Resume/Stop routine load job to LabelName such as [db.]name
5. Exclude final job when updating job
6. Catch all of exception when scheduling one job. The exception will not block the another jobs.
2019-04-28 10:33:50 +08:00