└── src ├── main ├── config │ ├── discovery.pinlater.local │ ├── example.queue_config.json │ ├── mysql.local.json │ ├── log4j.local.properties │ ├── redis.local.json │ ├── pinlater.redis.test.properties │ ├── pinlater.redis.local.properties │ ├── pinlater.test.properties │ └── pinlater.local.properties ├── scripts │ ├── run_server_local_mysql.sh │ ├── run_server_local_redis.sh │ └── run_server_common.sh ├── java │ └── com │ │ └── pinterest │ │ └── pinlater │ │ ├── commons │ │ ├── healthcheck │ │ │ ├── DummyHeartBeater.java │ │ │ ├── HeartBeater.java │ │ │ └── ServerTracker.java │ │ ├── jdbc │ │ │ ├── RowProcessor.java │ │ │ ├── Nulls.java │ │ │ └── SingleColumnRowProcessor.java │ │ ├── serviceframework │ │ │ └── ServiceShutdownHook.java │ │ ├── ostrich │ │ │ ├── OstrichAdminService.java │ │ │ └── StatTrackingEventListener.java │ │ ├── util │ │ │ ├── BytesUtil.java │ │ │ └── TimeUtils.java │ │ └── config │ │ │ └── ConfigFileServerSet.java │ │ ├── backends │ │ ├── redis │ │ │ ├── RedisHeartBeater.java │ │ │ ├── RedisConfigSchema.java │ │ │ ├── RedisClientHelper.java │ │ │ ├── EndPoint.java │ │ │ ├── JedisClientHelper.java │ │ │ ├── RedisUtils.java │ │ │ ├── RedisClientConfig.java │ │ │ ├── RedisPools.java │ │ │ └── RedisQueueMonitor.java │ │ ├── mysql │ │ │ ├── MySQLConfigSchema.java │ │ │ ├── MySQLDataSources.java │ │ │ ├── MySQLQueueMonitor.java │ │ │ ├── MySQLHealthMonitor.java │ │ │ └── MySQLQueries.java │ │ └── common │ │ │ ├── PinLaterJobDescriptor.java │ │ │ ├── BackendQueueMonitorBase.java │ │ │ └── PinLaterBackendUtils.java │ │ ├── example │ │ ├── PinLaterExampleJob.java │ │ └── PinLaterExampleWorker.java │ │ ├── QueueRateLimiter.java │ │ ├── client │ │ ├── RPCStatsLogger.java │ │ ├── PinLaterClient.java │ │ └── PinLaterClientTool.java │ │ ├── PinLaterBackendIface.java │ │ ├── PinLaterServer.java │ │ └── PinLaterQueueConfig.java └── assembly │ └── pinlater.xml └── test └── java └── com └── pinterest └── pinlater ├── backends ├── redis │ ├── LocalRedisChecker.java │ ├── DummyRandom.java │ └── RedisQueueMonitorTest.java ├── mysql │ ├── MySQLBackendUtilsTest.java │ ├── LocalMySQLChecker.java │ ├── MySQLQueueMonitorTest.java │ └── MySQLHealthMonitorTest.java └── common │ ├── PinLaterTestUtils.java │ ├── PinLaterJobDescriptorTest.java │ └── BackendQueueMonitorBaseTest.java └── PinLaterQueueConfigTest.java /src/main/config/discovery.pinlater.local: -------------------------------------------------------------------------------- 1 | 127.0.0.1:9010 -------------------------------------------------------------------------------- /src/main/config/example.queue_config.json: -------------------------------------------------------------------------------- 1 | { 2 | "queues": [ 3 | { 4 | "name": "pinlater_test_queue", 5 | "queueConfig": { 6 | "maxJobsPerSecond": 100 7 | } 8 | }, 9 | { 10 | "name": "pinlater_test_slow_queue", 11 | "queueConfig": { 12 | "maxJobsPerSecond": 0.1 13 | } 14 | }, 15 | { 16 | "name": "pinlater_test_paused_queue", 17 | "queueConfig": { 18 | "maxJobsPerSecond": 0 19 | } 20 | } 21 | ] 22 | } 23 | -------------------------------------------------------------------------------- /src/main/config/mysql.local.json: -------------------------------------------------------------------------------- 1 | { 2 | "pinlaterlocaldb001": { 3 | "master": { 4 | "host": "localhost", 5 | "port": 3306 6 | }, 7 | "slave": { 8 | "host": "localhost", 9 | "port": 3306 10 | }, 11 | "user": "root", 12 | "passwd": "" 13 | }, 14 | "pinlaterlocaldb002": { 15 | "master": { 16 | "host": "localhost", 17 | "port": 3306 18 | }, 19 | "slave": { 20 | "host": "localhost", 21 | "port": 3306 22 | }, 23 | "user": "root", 24 | "passwd": "" 25 | }, 26 | "pinlaterlocaldb003": { 27 | "master": { 28 | "host": "localhost", 29 | "port": 3306 30 | }, 31 | "slave": { 32 | "host": "localhost", 33 | "port": 3306 34 | }, 35 | "user": "root", 36 | "passwd": "", 37 | "dequeueOnly": true 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/test/java/com/pinterest/pinlater/backends/redis/LocalRedisChecker.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.pinlater.backends.redis; 2 | 3 | import com.google.common.collect.Maps; 4 | import redis.clients.jedis.Jedis; 5 | import redis.clients.jedis.exceptions.JedisConnectionException; 6 | 7 | import java.util.Map; 8 | 9 | /** 10 | * Class to check if redis is running on localhost. 11 | */ 12 | public class LocalRedisChecker { 13 | 14 | private static Map sIsRunning = Maps.newConcurrentMap(); 15 | 16 | public static boolean isRunning(int port) { 17 | if (sIsRunning.containsKey(port)) { 18 | return sIsRunning.get(port); 19 | } 20 | synchronized (sIsRunning) { 21 | Jedis conn = null; 22 | try { 23 | conn = new Jedis("localhost", port); 24 | conn.ping(); 25 | sIsRunning.put(port, true); 26 | } catch (JedisConnectionException e) { 27 | sIsRunning.put(port, false); 28 | } finally { 29 | conn.disconnect(); 30 | } 31 | return sIsRunning.get(port); 32 | } 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/config/log4j.local.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # log4j logging configuration. 17 | 18 | # root logger. 19 | log4j.rootLogger=INFO, CONSOLE 20 | 21 | log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender 22 | log4j.appender.CONSOLE.Threshold=INFO 23 | log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout 24 | log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} [%t] (%F:%L) %-5p %m%n -------------------------------------------------------------------------------- /src/main/scripts/run_server_local_mysql.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one or more 4 | # contributor license agreements. See the NOTICE file distributed with 5 | # this work for additional information regarding copyright ownership. 6 | # The ASF licenses this file to You under the Apache License, Version 2.0 7 | # (the "License"); you may not use this file except in compliance with 8 | # the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | 18 | # CHANGE THIS TO THE ROOT OF THE DIRECTORY CONTAINING EXTRACTED PINLATER JARS. 19 | export PINLATER_HOME_DIR=/var 20 | 21 | export LOG_PROPERTIES=log4j.local.properties 22 | export SERVER_CONFIG=pinlater.local.properties 23 | export BACKEND_CONFIG=mysql.local.json 24 | 25 | MY_DIR=`dirname $0` 26 | source $MY_DIR/run_server_common.sh 27 | -------------------------------------------------------------------------------- /src/main/scripts/run_server_local_redis.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one or more 4 | # contributor license agreements. See the NOTICE file distributed with 5 | # this work for additional information regarding copyright ownership. 6 | # The ASF licenses this file to You under the Apache License, Version 2.0 7 | # (the "License"); you may not use this file except in compliance with 8 | # the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | 18 | # CHANGE THIS TO THE ROOT OF THE DIRECTORY CONTAINING EXTRACTED PINLATER JARS. 19 | export PINLATER_HOME_DIR=/var 20 | 21 | export LOG_PROPERTIES=log4j.local.properties 22 | export SERVER_CONFIG=pinlater.redis.local.properties 23 | export BACKEND_CONFIG=redis.local.json 24 | 25 | MY_DIR=`dirname $0` 26 | source $MY_DIR/run_server_common.sh 27 | -------------------------------------------------------------------------------- /src/main/config/redis.local.json: -------------------------------------------------------------------------------- 1 | { 2 | "shards": [ 3 | { 4 | "name": "1", 5 | "shardConfig": { 6 | "master": { 7 | "host": "localhost", 8 | "port": 6379 9 | }, 10 | "slave": { 11 | "host": "localhost", 12 | "port": 6379 13 | } 14 | } 15 | }, 16 | { 17 | "name": "2", 18 | "shardConfig": { 19 | "master": { 20 | "host": "localhost", 21 | "port": 6379 22 | }, 23 | "slave": { 24 | "host": "localhost", 25 | "port": 6379 26 | } 27 | } 28 | }, 29 | { 30 | "name": "3", 31 | "shardConfig": { 32 | "master": { 33 | "host": "localhost", 34 | "port": 6379 35 | }, 36 | "slave": { 37 | "host": "localhost", 38 | "port": 6379 39 | }, 40 | "dequeueOnly": true 41 | } 42 | } 43 | ] 44 | } 45 | -------------------------------------------------------------------------------- /src/test/java/com/pinterest/pinlater/backends/mysql/MySQLBackendUtilsTest.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.pinlater.backends.mysql; 2 | 3 | import org.junit.Assert; 4 | import org.junit.Test; 5 | 6 | public class MySQLBackendUtilsTest { 7 | 8 | @Test 9 | public void testconstructDBName() { 10 | validateQueueName("test"); 11 | validateQueueName("test_queue"); 12 | validateQueueName("_test_"); 13 | } 14 | 15 | private void validateQueueName(String queueName) { 16 | String shardName = MySQLBackendUtils.constructShardName(1, 0); 17 | String dbName = MySQLBackendUtils.constructDBName(queueName, shardName); 18 | // Database 0's name should be backward compatible. 19 | Assert.assertEquals(MySQLBackendUtils.constructDBName(queueName, "1"), dbName); 20 | Assert.assertEquals(queueName, MySQLBackendUtils.queueNameFromDBName(dbName)); 21 | Assert.assertEquals(shardName, MySQLBackendUtils.shardNameFromDBName(dbName)); 22 | 23 | shardName = MySQLBackendUtils.constructShardName(1, 1); 24 | dbName = MySQLBackendUtils.constructDBName(queueName, shardName); 25 | Assert.assertEquals(queueName, MySQLBackendUtils.queueNameFromDBName(dbName)); 26 | Assert.assertEquals(shardName, MySQLBackendUtils.shardNameFromDBName(dbName)); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/commons/healthcheck/DummyHeartBeater.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.commons.healthcheck; 18 | 19 | public class DummyHeartBeater implements HeartBeater { 20 | 21 | public boolean heartBeatResult; 22 | 23 | public DummyHeartBeater(boolean heartBeatResult) { 24 | this.heartBeatResult = heartBeatResult; 25 | } 26 | 27 | @Override 28 | public boolean ping() { 29 | return heartBeatResult; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/commons/jdbc/RowProcessor.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.commons.jdbc; 18 | 19 | import java.io.IOException; 20 | import java.sql.ResultSet; 21 | import java.sql.SQLException; 22 | 23 | /** 24 | * An simple interface for processing a single row in ResultSet. 25 | * @param the type of the returned object by processing a single row in ResultSet. 26 | */ 27 | public interface RowProcessor { 28 | 29 | T process(ResultSet rs) throws IOException, SQLException; 30 | } 31 | 32 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/commons/healthcheck/HeartBeater.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.commons.healthcheck; 18 | 19 | /** 20 | * Interface that sends a command to check health of a server. 21 | */ 22 | public interface HeartBeater { 23 | 24 | /** 25 | * This function is used to ping the server. 26 | * 27 | * It should return True if the server is live. Otherwise either return False or throw an 28 | * exception to indicate a failed heartbeat. 29 | */ 30 | public boolean ping() throws Exception; 31 | } 32 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/commons/jdbc/Nulls.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.commons.jdbc; 18 | 19 | /** 20 | * An enum class to make it easy to tell the underlying type of NULL column. 21 | */ 22 | public enum Nulls { 23 | // string or clob 24 | NULL_STRING, 25 | NULL_BOOLEAN, 26 | NULL_BYTE, 27 | NULL_SHORT, 28 | NULL_INTEGER, 29 | NULL_LONG, 30 | NULL_FLOAT, 31 | NULL_DOUBLE, 32 | // blob 33 | NULL_BYTE_ARRAY, 34 | // no java.sql.Date, we only support java.sql.Timestamp 35 | NULL_TIMESTAMP, 36 | NULL_BIGDECIMAL 37 | } 38 | 39 | -------------------------------------------------------------------------------- /src/main/config/pinlater.redis.test.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | BACKEND_CONNECTION_MAX_WAIT_MILLIS=5000 17 | BACKEND_CONNECTIONS_PER_SHARD=10 18 | BACKEND_DEQUEUE_CONCURRENCY_PER_QUEUE_PER_SHARD=10 19 | BACKEND_MONITOR_JOB_CLAIMED_TIMEOUT_SECONDS=180 20 | BACKEND_MONITOR_JOB_FAILED_GC_TTL_HOURS=2 21 | BACKEND_MONITOR_JOB_SUCCEEDED_GC_TTL_HOURS=1 22 | BACKEND_MONITOR_THREAD_DELAY_SECONDS=120 23 | BACKEND_MONITOR_UPDATE_MAX_SIZE=1000 24 | BACKEND_NUM_AUTO_RETRIES=2 25 | BACKEND_QUERY_PARALLELISM=5 26 | BACKEND_SOCKET_TIMEOUT_SECONDS=1 27 | MAX_CONCURRENT_REQUESTS=5000 28 | NUM_PRIORITY_LEVELS=3 29 | PINLATER_BACKEND=redis 30 | QUEUE_CONFIG_FILE_PATH= 31 | OSTRICH_PORT=9999 32 | SERVER_CONN_MAX_IDLE_TIME_MINUTES=5 33 | SERVER_SET_ENABLED=false 34 | THRIFT_PORT=9010 -------------------------------------------------------------------------------- /src/main/config/pinlater.redis.local.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | BACKEND_CONNECTION_MAX_WAIT_MILLIS=5000 17 | BACKEND_CONNECTIONS_PER_SHARD=10 18 | BACKEND_DEQUEUE_CONCURRENCY_PER_QUEUE_PER_SHARD=10 19 | BACKEND_MONITOR_JOB_CLAIMED_TIMEOUT_SECONDS=180 20 | BACKEND_MONITOR_JOB_FAILED_GC_TTL_HOURS=2 21 | BACKEND_MONITOR_JOB_SUCCEEDED_GC_TTL_HOURS=1 22 | BACKEND_MONITOR_THREAD_DELAY_SECONDS=120 23 | BACKEND_MONITOR_UPDATE_MAX_SIZE=1000 24 | BACKEND_NUM_AUTO_RETRIES=2 25 | BACKEND_QUERY_PARALLELISM=5 26 | BACKEND_SOCKET_TIMEOUT_SECONDS=1 27 | MAX_CONCURRENT_REQUESTS=5000 28 | NUM_PRIORITY_LEVELS=3 29 | PINLATER_BACKEND=redis 30 | QUEUE_CONFIG_FILE_PATH=example.queue_config.json 31 | OSTRICH_PORT=9999 32 | SERVER_CONN_MAX_IDLE_TIME_MINUTES=5 33 | SERVER_SET_ENABLED=true 34 | SERVER_SET_PATH=discovery.pinlater.local 35 | THRIFT_PORT=9010 -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/redis/RedisHeartBeater.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.redis; 18 | 19 | import com.google.common.base.Preconditions; 20 | import com.pinterest.pinlater.commons.healthcheck.HeartBeater; 21 | 22 | 23 | public class RedisHeartBeater implements HeartBeater { 24 | 25 | private final RC redisClient; 26 | private final RedisClientHelper redisClientHelper; 27 | 28 | public RedisHeartBeater(RedisClientHelper redisClientHelper, RC redisClient) { 29 | this.redisClient = Preconditions.checkNotNull(redisClient); 30 | this.redisClientHelper = redisClientHelper; 31 | } 32 | 33 | @Override 34 | public boolean ping() throws Exception { 35 | return redisClientHelper.clientPing(redisClient); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/main/config/pinlater.test.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | BACKEND_CONNECTION_MAX_WAIT_MILLIS=5000 17 | BACKEND_CONNECTIONS_PER_SHARD=10 18 | BACKEND_DEQUEUE_CONCURRENCY_PER_QUEUE_PER_SHARD=4 19 | BACKEND_MONITOR_JOB_CLAIMED_TIMEOUT_SECONDS=180 20 | BACKEND_MONITOR_JOB_FAILED_GC_TTL_HOURS=2 21 | BACKEND_MONITOR_JOB_SUCCEEDED_GC_TTL_HOURS=1 22 | BACKEND_MONITOR_THREAD_DELAY_SECONDS=120 23 | BACKEND_MONITOR_UPDATE_MAX_SIZE=5000 24 | BACKEND_NUM_AUTO_RETRIES=3 25 | BACKEND_QUERY_PARALLELISM=5 26 | BACKEND_SOCKET_TIMEOUT_SECONDS=5 27 | MAX_CONCURRENT_REQUESTS=5000 28 | NUM_PRIORITY_LEVELS=3 29 | MYSQL_COUNT_LIMIT=100000 30 | MYSQL_NUM_DB_PER_QUEUE=3 31 | QUEUE_CONFIG_FILE_PATH= 32 | OSTRICH_PORT=9999 33 | SERVER_CONN_MAX_IDLE_TIME_MINUTES=5 34 | SERVER_SET_ENABLED=false 35 | SHARD_ALLOWED_HOSTNAME_PREFIX=pinlaterlocaldb 36 | THRIFT_PORT=9010 -------------------------------------------------------------------------------- /src/test/java/com/pinterest/pinlater/backends/common/PinLaterTestUtils.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.common; 18 | 19 | import com.pinterest.pinlater.PinLaterBackendBase; 20 | import com.pinterest.pinlater.thrift.PinLaterGetJobCountRequest; 21 | import com.pinterest.pinlater.thrift.PinLaterJobState; 22 | 23 | public class PinLaterTestUtils { 24 | 25 | /** 26 | * Convenient interface to getJobCount to make testing a little less verbose. 27 | * 28 | * @param queueName Name of queue to count jobs in. 29 | * @param jobState State of jobs to look for. 30 | * @return An int as job count. 31 | */ 32 | public static int getJobCount(PinLaterBackendBase backend, final String queueName, 33 | final PinLaterJobState jobState) { 34 | return backend.getJobCount(new PinLaterGetJobCountRequest(queueName, 35 | jobState)).get().intValue(); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/example/PinLaterExampleJob.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 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 3 | * 4 | * http://www.apache.org/licenses/LICENSE-2.0 5 | * 6 | * 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. 7 | * 8 | */ 9 | 10 | package com.pinterest.pinlater.example; 11 | 12 | import com.pinterest.pinlater.thrift.PinLaterJob; 13 | import org.slf4j.Logger; 14 | import org.slf4j.LoggerFactory; 15 | 16 | import java.nio.ByteBuffer; 17 | 18 | /** 19 | * An example PinLater job 20 | */ 21 | public class PinLaterExampleJob { 22 | 23 | private static final Logger LOG = LoggerFactory.getLogger(PinLaterExampleJob.class); 24 | public static final String QUEUE_NAME = "test_queue"; 25 | 26 | private String logData; 27 | 28 | public PinLaterExampleJob(String logData) { 29 | this.logData = logData; 30 | } 31 | 32 | /** 33 | * Build a PinLaterJob object that can be used to build an enqueue request. 34 | */ 35 | public PinLaterJob buildJob() { 36 | PinLaterJob job = new PinLaterJob(ByteBuffer.wrap(logData.getBytes())); 37 | job.setNumAttemptsAllowed(10); 38 | return job; 39 | } 40 | 41 | public static void process(String logData) throws Exception{ 42 | LOG.info("PinLaterExampleJob: {}", logData); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /src/main/config/pinlater.local.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # PASSWORD_HASH required by the deleteQueue API. It's computed by SHA-256(password + SALT). Check 17 | # out PinLaterBackendUtils.java for more details 18 | ADMIN_PASSWORD_HASH= 19 | BACKEND_CONNECTION_MAX_WAIT_MILLIS=5000 20 | BACKEND_CONNECTIONS_PER_SHARD=10 21 | BACKEND_DEQUEUE_CONCURRENCY_PER_QUEUE_PER_SHARD=4 22 | BACKEND_MONITOR_JOB_CLAIMED_TIMEOUT_SECONDS=180 23 | BACKEND_MONITOR_JOB_FAILED_GC_TTL_HOURS=2 24 | BACKEND_MONITOR_JOB_SUCCEEDED_GC_TTL_HOURS=1 25 | BACKEND_MONITOR_THREAD_DELAY_SECONDS=120 26 | BACKEND_MONITOR_UPDATE_MAX_SIZE=5000 27 | BACKEND_NUM_AUTO_RETRIES=3 28 | BACKEND_QUERY_PARALLELISM=5 29 | BACKEND_SOCKET_TIMEOUT_SECONDS=5 30 | MAX_CONCURRENT_REQUESTS=5000 31 | NUM_PRIORITY_LEVELS=3 32 | MYSQL_COUNT_LIMIT=100000 33 | MYSQL_NUM_DB_PER_QUEUE=4 34 | QUEUE_CONFIG_FILE_PATH=example.queue_config.json 35 | OSTRICH_PORT=9999 36 | SERVER_CONN_MAX_IDLE_TIME_MINUTES=5 37 | SERVER_SET_ENABLED=true 38 | SERVER_SET_PATH=discovery.pinlater.local 39 | SHARD_ALLOWED_HOSTNAME_PREFIX=pinlaterlocaldb 40 | THRIFT_PORT=9010 -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/redis/RedisConfigSchema.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.redis; 18 | 19 | import org.codehaus.jackson.map.ObjectMapper; 20 | 21 | import java.io.IOException; 22 | import java.io.InputStream; 23 | import java.util.List; 24 | 25 | /** 26 | * Helper class used by the PinLaterRedisBackend that describes the Redis config schema 27 | * (list of shards and the machines they map to). 28 | */ 29 | public class RedisConfigSchema { 30 | 31 | private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); 32 | 33 | public List shards; 34 | 35 | public static class Shard { 36 | 37 | public String name; 38 | public ShardConfig shardConfig; 39 | } 40 | 41 | public static class ShardConfig { 42 | 43 | public EndPoint master; 44 | public EndPoint slave; 45 | public boolean dequeueOnly = false; 46 | } 47 | 48 | public static class EndPoint { 49 | 50 | public String host; 51 | public int port; 52 | } 53 | 54 | public static RedisConfigSchema read(InputStream json) throws IOException { 55 | return OBJECT_MAPPER.readValue(json, RedisConfigSchema.class); 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/test/java/com/pinterest/pinlater/backends/mysql/LocalMySQLChecker.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.mysql; 18 | 19 | import com.pinterest.pinlater.commons.jdbc.JdbcUtils; 20 | 21 | import java.sql.Connection; 22 | import java.sql.DriverManager; 23 | import java.sql.SQLException; 24 | import java.util.concurrent.atomic.AtomicBoolean; 25 | 26 | /** 27 | * Class to check if mysql is running on localhost. 28 | */ 29 | public class LocalMySQLChecker { 30 | 31 | private static AtomicBoolean sChecked = new AtomicBoolean(false); 32 | private static AtomicBoolean sIsRunning = new AtomicBoolean(false); 33 | 34 | public static boolean isRunning() { 35 | if (sChecked.get()) { 36 | return sIsRunning.get(); 37 | } 38 | synchronized (LocalMySQLChecker.class) { 39 | Connection conn = null; 40 | try { 41 | DriverManager.getConnection("jdbc:mysql://localhost:3306", "root", ""); 42 | sIsRunning.set(true); 43 | } catch (SQLException e) { 44 | sIsRunning.set(false); 45 | JdbcUtils.closeConnection(conn); 46 | } 47 | sChecked.set(true); 48 | return sIsRunning.get(); 49 | } 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /src/test/java/com/pinterest/pinlater/backends/redis/DummyRandom.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.redis; 18 | 19 | import com.google.common.base.Preconditions; 20 | import org.apache.commons.lang.NotImplementedException; 21 | 22 | import java.util.Random; 23 | 24 | /** 25 | * Dummy Random class which only provides configurable nextInt() and nextInt(int i) API. 26 | */ 27 | public class DummyRandom extends Random { 28 | 29 | private int nextInt = 0; 30 | 31 | public void setNextInt(int i) { 32 | Preconditions.checkArgument(i >= 0); 33 | nextInt = i; 34 | } 35 | 36 | public int nextInt() { 37 | return nextInt; 38 | } 39 | 40 | public void nextBytes(byte[] bytes) { 41 | throw new NotImplementedException(); 42 | } 43 | 44 | public int nextInt(int i) { 45 | return nextInt; 46 | } 47 | 48 | public long nextLong() { 49 | throw new NotImplementedException(); 50 | } 51 | 52 | public boolean nextBoolean() { 53 | throw new NotImplementedException(); 54 | } 55 | 56 | public float nextFloat() { 57 | throw new NotImplementedException(); 58 | } 59 | 60 | public double nextDouble() { 61 | throw new NotImplementedException(); 62 | } 63 | 64 | public synchronized double nextGaussian() { 65 | throw new NotImplementedException(); 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/QueueRateLimiter.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater; 18 | 19 | import com.google.common.util.concurrent.RateLimiter; 20 | 21 | /** 22 | * Simple wrapper class for Guava's RateLimiter. This is necessary because the Guava 23 | * implementation does not allow rate to be zero, something which we need. 24 | */ 25 | public final class QueueRateLimiter { 26 | 27 | private QueueRateLimiter() {} 28 | 29 | public static interface IFace { 30 | 31 | public boolean allowDequeue(int numJobs); 32 | 33 | public double getRate(); 34 | } 35 | 36 | public static IFace create(double maxRequestsPerSecond) { 37 | if (maxRequestsPerSecond <= 0.0) { 38 | return ALLOW_NONE; 39 | } 40 | 41 | final RateLimiter rateLimiter = RateLimiter.create(maxRequestsPerSecond); 42 | return new IFace() { 43 | @Override 44 | public boolean allowDequeue(int numJobs) { 45 | return rateLimiter.tryAcquire(numJobs); 46 | } 47 | 48 | @Override 49 | public double getRate() { 50 | return rateLimiter.getRate(); 51 | } 52 | }; 53 | } 54 | 55 | private static final IFace ALLOW_NONE = new IFace() { 56 | @Override 57 | public boolean allowDequeue(int numJobs) { 58 | return false; 59 | } 60 | 61 | @Override 62 | public double getRate() { 63 | return 0.0; 64 | } 65 | }; 66 | } 67 | -------------------------------------------------------------------------------- /src/main/assembly/pinlater.xml: -------------------------------------------------------------------------------- 1 | 2 | 13 | 14 | bin 15 | 16 | 17 | tar.gz 18 | 19 | 20 | 21 | src/main/scripts 22 | scripts 23 | 24 | run_server*.sh 25 | 26 | 27 | 28 | target 29 | 30 | 31 | pinlater*.jar 32 | 33 | 34 | 35 | target/classes 36 | 37 | 38 | *.json 39 | *.properties 40 | discovery.* 41 | 42 | 43 | 44 | 45 | 46 | 47 | lib 48 | 49 | org.slf4j:slf4j-jdk14 50 | 51 | true 52 | false 53 | runtime 54 | 55 | 56 | 57 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/redis/RedisClientHelper.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.redis; 18 | 19 | /** 20 | * RedisClientBuilders abstract away redis client's from RedisPools. 21 | * 22 | * RedisClientBuilders MUST be able to handle concurrent requests. 23 | * 24 | * @param RedisClient type, for example JedisPool 25 | */ 26 | public interface RedisClientHelper { 27 | 28 | /** 29 | * createClient should create a new client of type RC (Redis Client) 30 | * @param redisClientConfig A RedisClientConfig representing the parameters the client utilize 31 | * @param endPoint The host that this client should connect to 32 | * @return A client adhering to redisClientConfig connected to endPoint 33 | */ 34 | RC createClient(RedisClientConfig redisClientConfig, EndPoint endPoint); 35 | 36 | /** 37 | * destroyClient should cleanup the redis client 38 | * @param redisClient The client to be cleaned up. 39 | */ 40 | void destroyClient(RC redisClient); 41 | 42 | /** 43 | * clientPing should ping the redis instance connected to this client. 44 | * 45 | * clientPing is used in the RedisHeartBeater to ensure redis instance liveliness. 46 | * 47 | * @param redisClient The redisClient to use to ping redis 48 | * @return true if ping was successful and replied PONG, false otherwise 49 | */ 50 | boolean clientPing(RC redisClient); 51 | } 52 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/redis/EndPoint.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.redis; 18 | 19 | public class EndPoint { 20 | 21 | public String host; 22 | public int port; 23 | public int socket_timeout; 24 | 25 | public EndPoint(String host, int port, int socket_timeout) { 26 | this.host = host; 27 | this.port = port; 28 | this.socket_timeout = socket_timeout; 29 | } 30 | 31 | public EndPoint(EndPoint endpoint) { 32 | this.host = endpoint.host; 33 | this.port = endpoint.port; 34 | this.socket_timeout = endpoint.socket_timeout; 35 | } 36 | 37 | @Override 38 | public String toString() { 39 | return "EndPoint{" 40 | + "host='" + host + '\'' 41 | + ", port=" + port 42 | + ", socket_timeout=" + socket_timeout 43 | + '}'; 44 | } 45 | 46 | @Override 47 | public boolean equals(Object o) { 48 | if (this == o) { 49 | return true; 50 | } 51 | if (!(o instanceof EndPoint)) { 52 | return false; 53 | } 54 | 55 | EndPoint endPoint = (EndPoint) o; 56 | 57 | if (port == endPoint.port 58 | && host.equals(endPoint.host) 59 | && socket_timeout == endPoint.socket_timeout) { 60 | return true; 61 | } 62 | 63 | return false; 64 | } 65 | 66 | @Override 67 | public int hashCode() { 68 | int result = (host != null) ? host.hashCode() : 0; 69 | result = 31 * result + port; 70 | result = 31 * result + socket_timeout; 71 | return result; 72 | } 73 | } -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/client/RPCStatsLogger.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.client; 18 | 19 | import com.google.common.base.Preconditions; 20 | import com.twitter.util.Duration; 21 | import org.slf4j.Logger; 22 | import org.slf4j.LoggerFactory; 23 | 24 | /** 25 | * A simple utility to keep track of and periodically log QPS and latency for a client or server. 26 | */ 27 | public class RPCStatsLogger { 28 | 29 | private static final Logger LOG = LoggerFactory.getLogger(RPCStatsLogger.class); 30 | 31 | private final int logIntervalSeconds; 32 | private long queriesIssued = 0; 33 | private long totalLatencyMicros = 0; 34 | private long lastLogTimestampMillis = 0; 35 | 36 | public RPCStatsLogger(int logIntervalSeconds) { 37 | this.logIntervalSeconds = Preconditions.checkNotNull(logIntervalSeconds); 38 | } 39 | 40 | /** 41 | * Call this when each request completes. 42 | * 43 | * @param latency Latency for that request. 44 | */ 45 | public synchronized void requestComplete(Duration latency) { 46 | queriesIssued++; 47 | totalLatencyMicros += latency.inMicroseconds(); 48 | 49 | long currentTimeMillis = System.currentTimeMillis(); 50 | if (currentTimeMillis >= lastLogTimestampMillis + logIntervalSeconds * 1000) { 51 | long qps = queriesIssued * 1000 / (currentTimeMillis - lastLogTimestampMillis); 52 | double avgLatencyMillis = totalLatencyMicros / (queriesIssued * 1000.0); 53 | LOG.info(String.format("QPS: %d Avg Latency (ms): %.5f", qps, avgLatencyMillis)); 54 | 55 | queriesIssued = 0; 56 | totalLatencyMicros = 0; 57 | lastLogTimestampMillis = currentTimeMillis; 58 | } 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /src/main/scripts/run_server_common.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one or more 4 | # contributor license agreements. See the NOTICE file distributed with 5 | # this work for additional information regarding copyright ownership. 6 | # The ASF licenses this file to You under the Apache License, Version 2.0 7 | # (the "License"); you may not use this file except in compliance with 8 | # the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | 18 | JAVA_HOME=/usr 19 | JAVA=${JAVA_HOME}/bin/java 20 | 21 | LOG_DIR=${PINLATER_HOME_DIR}/log 22 | RUN_DIR=${PINLATER_HOME_DIR}/run 23 | PIDFILE=${RUN_DIR}/pinlater.pid 24 | 25 | LIB=${PINLATER_HOME_DIR} 26 | CP=${LIB}:${LIB}/*:${LIB}/lib/* 27 | 28 | DAEMON_OPTS="-server -Xmx5G -Xms5G -XX:NewSize=3G -verbosegc -Xloggc:${LOG_DIR}/gc.log \ 29 | -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=100 -XX:GCLogFileSize=2M \ 30 | -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintGCDateStamps -XX:+PrintClassHistogram \ 31 | -XX:+HeapDumpOnOutOfMemoryError -XX:+UseConcMarkSweepGC -XX:+UseParNewGC \ 32 | -XX:ErrorFile=${LOG_DIR}/jvm_error.log \ 33 | -cp ${CP} -Dlog4j.configuration=${LOG_PROPERTIES} -Dserver_config=${SERVER_CONFIG} \ 34 | -Dbackend_config=${BACKEND_CONFIG} com.pinterest.pinlater.PinLaterServer" 35 | 36 | function server_start { 37 | echo -n "Starting ${NAME}: " 38 | mkdir -p ${LOG_DIR} 39 | chmod 755 ${LOG_DIR} 40 | mkdir -p ${RUN_DIR} 41 | touch ${PIDFILE} 42 | chmod 755 ${RUN_DIR} 43 | start-stop-daemon --start --quiet --umask 007 --pidfile ${PIDFILE} --make-pidfile \ 44 | --exec ${JAVA} -- ${DAEMON_OPTS} 2>&1 < /dev/null & 45 | echo "${NAME} started." 46 | } 47 | 48 | function server_stop { 49 | echo -n "Stopping ${NAME}: " 50 | start-stop-daemon --stop --quiet --pidfile ${PIDFILE} --retry=TERM/30/KILL/5 51 | echo "${NAME} stopped." 52 | rm -f ${RUN_DIR}/* 53 | } 54 | 55 | case "$1" in 56 | 57 | start) 58 | server_start 59 | ;; 60 | 61 | stop) 62 | server_stop 63 | ;; 64 | 65 | restart) 66 | server_stop 67 | server_start 68 | ;; 69 | 70 | esac 71 | 72 | exit 0 73 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/commons/jdbc/SingleColumnRowProcessor.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.commons.jdbc; 18 | 19 | import java.sql.ResultSet; 20 | import java.sql.SQLException; 21 | 22 | /** 23 | * An utility class to make it easy to retrieve a single column from the ResultSet. 24 | * @param the return type of the column to be processed. 25 | */ 26 | public class SingleColumnRowProcessor implements RowProcessor { 27 | 28 | private Class mExpectedType; 29 | private int mIndex; 30 | 31 | /** 32 | * Constructor. 33 | * 34 | * @param expectedType the class of the expected return type of the target column in ResultSet. 35 | */ 36 | public SingleColumnRowProcessor(Class expectedType) { 37 | this(expectedType, 1); 38 | } 39 | 40 | /** 41 | * Constructor. Use this constructor if the target column is not the first column in the 42 | * rows of the ResultSet. 43 | * 44 | * @param expectedType the class of the expected return type of the target column in ResultSet. 45 | * @param index the index of the column in the ResultSet. 46 | */ 47 | public SingleColumnRowProcessor(Class expectedType, int index) { 48 | this.mExpectedType = expectedType; 49 | this.mIndex = index; 50 | } 51 | 52 | /** 53 | * The actual implementation of retrieving a column from the current row in the ResultSet. 54 | * 55 | * @param rs the ResultSet from which to process the current row. 56 | * @return the corresponding value from processing the current row in the ResultSet. 57 | * @throws SQLException 58 | */ 59 | public T process(ResultSet rs) throws SQLException { 60 | // Extract column value from JDBC ResultSet. 61 | return (T) JdbcUtils.getResultSetValue(rs, this.mIndex, this.mExpectedType); 62 | } 63 | 64 | } 65 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/mysql/MySQLConfigSchema.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.mysql; 18 | 19 | import org.codehaus.jackson.JsonNode; 20 | import org.codehaus.jackson.map.ObjectMapper; 21 | 22 | import java.io.InputStream; 23 | import java.util.ArrayList; 24 | import java.util.Iterator; 25 | import java.util.List; 26 | 27 | /** 28 | * Helper class used by the PinLaterMySQLBackend that describes the MySQL config schema 29 | * (list of shards and the machines they map to). 30 | */ 31 | public class MySQLConfigSchema { 32 | 33 | private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); 34 | 35 | public List shards; 36 | 37 | public static class Shard { 38 | 39 | public int id; 40 | public ShardConfig shardConfig; 41 | } 42 | 43 | public static class ShardConfig { 44 | 45 | public EndPoint master; 46 | public EndPoint slave; 47 | public String user; 48 | public String passwd; 49 | public boolean dequeueOnly = false; 50 | } 51 | 52 | public static class EndPoint { 53 | 54 | public String host; 55 | public int port; 56 | } 57 | 58 | public static MySQLConfigSchema read(InputStream json, 59 | String shardNamePrefix) throws Exception { 60 | JsonNode shardsJson = OBJECT_MAPPER.readTree(json); 61 | 62 | MySQLConfigSchema mySQLConfigSchema = new MySQLConfigSchema(); 63 | mySQLConfigSchema.shards = new ArrayList(); 64 | 65 | Iterator shardNameIter = shardsJson.getFieldNames(); 66 | while (shardNameIter.hasNext()) { 67 | String shardName = shardNameIter.next(); 68 | if (shardName.startsWith(shardNamePrefix)) { 69 | Shard shard = new Shard(); 70 | shard.id = Integer.parseInt(shardName.replaceFirst(shardNamePrefix, "")); 71 | shard.shardConfig = OBJECT_MAPPER.readValue(shardsJson.get(shardName), ShardConfig.class); 72 | mySQLConfigSchema.shards.add(shard); 73 | } 74 | } 75 | return mySQLConfigSchema; 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/test/java/com/pinterest/pinlater/backends/common/PinLaterJobDescriptorTest.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.common; 18 | 19 | import org.junit.Assert; 20 | import org.junit.Test; 21 | 22 | public class PinLaterJobDescriptorTest { 23 | 24 | @Test 25 | public void testBasicFunctionality() { 26 | PinLaterJobDescriptor jobDesc1 = new PinLaterJobDescriptor("job_desc_test", "1", 2, 12345L); 27 | Assert.assertEquals("job_desc_test", jobDesc1.getQueueName()); 28 | Assert.assertEquals("1", jobDesc1.getShardName()); 29 | Assert.assertEquals(2, jobDesc1.getPriority()); 30 | Assert.assertEquals(12345L, jobDesc1.getLocalId()); 31 | 32 | PinLaterJobDescriptor jobDesc2 = new PinLaterJobDescriptor(jobDesc1.toString()); 33 | Assert.assertEquals(jobDesc1.getQueueName(), jobDesc2.getQueueName()); 34 | Assert.assertEquals(jobDesc1.getShardName(), jobDesc2.getShardName()); 35 | Assert.assertEquals(jobDesc1.getPriority(), jobDesc2.getPriority()); 36 | Assert.assertEquals(jobDesc1.getLocalId(), jobDesc2.getLocalId()); 37 | } 38 | 39 | @Test(expected = IllegalArgumentException.class) 40 | public void testQueueNameEmptyValidation() { 41 | new PinLaterJobDescriptor("", "1", 2, 12345L); 42 | } 43 | 44 | @Test(expected = NullPointerException.class) 45 | public void testQueueNameNullValidation() { 46 | new PinLaterJobDescriptor(null, "1", 2, 12345L); 47 | } 48 | 49 | @Test(expected = IllegalArgumentException.class) 50 | public void testShardNameEmptyValidation() { 51 | new PinLaterJobDescriptor("job_desc_test", "", 2, 12345L); 52 | } 53 | 54 | @Test(expected = NullPointerException.class) 55 | public void testShardNameNullValidation() { 56 | new PinLaterJobDescriptor("job_desc_test", null, 2, 12345L); 57 | } 58 | 59 | @Test(expected = IllegalArgumentException.class) 60 | public void testPriorityValidation() { 61 | new PinLaterJobDescriptor("job_desc_test", "1", -2, 12345L); 62 | } 63 | 64 | @Test(expected = IllegalArgumentException.class) 65 | public void testLocalIdValidation() { 66 | new PinLaterJobDescriptor("job_desc_test", "1", 2, -12345L); 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/commons/serviceframework/ServiceShutdownHook.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.commons.serviceframework; 18 | 19 | import com.twitter.finagle.builder.Server; 20 | import com.twitter.util.Duration; 21 | import org.slf4j.Logger; 22 | import org.slf4j.LoggerFactory; 23 | 24 | /** 25 | * This class is a utility to gracefully shutdown your finagle server. Upon jvm being shutdown, it 26 | * first stops the server from accepting new connections, and then wait till either the grace 27 | * period ends or all the requests drained with all the existing established connections, whichever 28 | * is earlier. 29 | */ 30 | public class ServiceShutdownHook { 31 | 32 | private static final Logger LOG = LoggerFactory.getLogger(ServiceShutdownHook.class); 33 | 34 | // default grace period is 10 seconds 35 | public static final long DEFAULT_GRACE_PERIOD_MILLIS = 10000L; 36 | 37 | /** 38 | * Turn on graceful shutdown on the finagle server passed in with the grace period of 10 seconds. 39 | * 40 | * @param server the finagle server instance built by finagle ServerBuilder 41 | */ 42 | public static void register(final Server server) { 43 | register(server, Duration.fromMilliseconds(DEFAULT_GRACE_PERIOD_MILLIS)); 44 | } 45 | 46 | /** 47 | * Turn on graceful shutdown on the finagle server passed in with the grace period passed in. 48 | * 49 | * @param server the finagle server instance built by finagle ServerBuilder. 50 | * @param gracePeriod the time period the shutdown process will wait for till the existing 51 | * requests drain. If the existing requests are not being drain after grace 52 | * period expires, the server will be forcefully shutdown. 53 | */ 54 | public static void register(final Server server, final Duration gracePeriod) { 55 | Runtime.getRuntime().addShutdownHook(new Thread() { 56 | @Override 57 | public void run() { 58 | LOG.info("Try to shut down the server gracefully: {}", gracePeriod); 59 | server.close(gracePeriod); 60 | LOG.info("Finished server graceful shutdown"); 61 | } 62 | }); 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/common/PinLaterJobDescriptor.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.common; 18 | 19 | import com.google.common.base.Preconditions; 20 | import com.twitter.common.base.MorePreconditions; 21 | 22 | /** 23 | * Helper class used by the different backends to encapsulate routines dealing 24 | * with job descriptors. 25 | */ 26 | public class PinLaterJobDescriptor { 27 | 28 | private final String queueName; 29 | private final String shardName; 30 | private final int priority; 31 | private final long localId; 32 | private final String formattedString; 33 | 34 | public PinLaterJobDescriptor(String queueName, String shardName, int priority, long localId) { 35 | this.queueName = MorePreconditions.checkNotBlank(queueName); 36 | this.shardName = MorePreconditions.checkNotBlank(shardName); 37 | Preconditions.checkArgument(priority >= 0); 38 | this.priority = priority; 39 | Preconditions.checkArgument(localId >= 0); 40 | this.localId = localId; 41 | 42 | // NOTE: Queue names are restricted to alphanumeric + underscores by API contract, enforced 43 | // above the backend layer. We don't re-enforce in this class for performance reasons. 44 | this.formattedString = String.format("%s:s%s:p%d:%d", queueName, shardName, priority, localId); 45 | } 46 | 47 | public PinLaterJobDescriptor(String jobDescriptor) { 48 | this.formattedString = MorePreconditions.checkNotBlank(jobDescriptor); 49 | 50 | String[] tokens = jobDescriptor.split(":"); 51 | Preconditions.checkArgument(tokens.length == 4); 52 | 53 | this.queueName = tokens[0]; 54 | this.shardName = tokens[1].substring(1); 55 | this.priority = Integer.parseInt(tokens[2].substring(1)); 56 | this.localId = Long.parseLong(tokens[3]); 57 | 58 | } 59 | 60 | public String toString() { 61 | return formattedString; 62 | } 63 | 64 | public String getQueueName() { 65 | return queueName; 66 | } 67 | 68 | public String getShardName() { 69 | return shardName; 70 | } 71 | 72 | public int getPriority() { 73 | return priority; 74 | } 75 | 76 | public long getLocalId() { 77 | return localId; 78 | } 79 | } 80 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/PinLaterBackendIface.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater; 18 | 19 | import com.pinterest.pinlater.thrift.PinLaterCheckpointJobsRequest; 20 | import com.pinterest.pinlater.thrift.PinLaterDeleteJobsRequest; 21 | import com.pinterest.pinlater.thrift.PinLaterDequeueRequest; 22 | import com.pinterest.pinlater.thrift.PinLaterDequeueResponse; 23 | import com.pinterest.pinlater.thrift.PinLaterEnqueueRequest; 24 | import com.pinterest.pinlater.thrift.PinLaterEnqueueResponse; 25 | import com.pinterest.pinlater.thrift.PinLaterGetJobCountRequest; 26 | import com.pinterest.pinlater.thrift.PinLaterJobAckRequest; 27 | import com.pinterest.pinlater.thrift.PinLaterJobInfo; 28 | import com.pinterest.pinlater.thrift.PinLaterLookupJobRequest; 29 | import com.pinterest.pinlater.thrift.PinLaterRetryFailedJobsRequest; 30 | import com.pinterest.pinlater.thrift.PinLaterScanJobsRequest; 31 | import com.pinterest.pinlater.thrift.PinLaterScanJobsResponse; 32 | 33 | import com.twitter.util.Future; 34 | 35 | import java.util.Map; 36 | import java.util.Set; 37 | 38 | /** 39 | * PinLater backend API. Currently, directly mirrors the PinLater thrift service API. 40 | * Backend implementations should be async and non-blocking. 41 | */ 42 | public interface PinLaterBackendIface { 43 | 44 | Future createQueue(String name); 45 | 46 | Future deleteQueue(String name, String password); 47 | 48 | Future enqueueJobs(PinLaterEnqueueRequest request); 49 | 50 | Future dequeueJobs(String source, PinLaterDequeueRequest request); 51 | 52 | Future ackDequeuedJobs(PinLaterJobAckRequest request); 53 | 54 | Future checkpointJobs(String source, PinLaterCheckpointJobsRequest request); 55 | 56 | Future> lookupJobs(PinLaterLookupJobRequest request); 57 | 58 | Future getJobCount(PinLaterGetJobCountRequest request); 59 | 60 | Future> getQueueNames(); 61 | 62 | Future scanJobs(PinLaterScanJobsRequest request); 63 | 64 | Future retryFailedJobs(PinLaterRetryFailedJobsRequest request); 65 | 66 | Future deleteJobs(PinLaterDeleteJobsRequest request); 67 | } -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/redis/JedisClientHelper.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.redis; 18 | 19 | import com.twitter.util.Function; 20 | import redis.clients.jedis.Jedis; 21 | import redis.clients.jedis.JedisPool; 22 | import redis.clients.jedis.JedisPoolConfig; 23 | 24 | /** 25 | * JedisClientHelper creates/destroys Jedis Client Pools. 26 | * 27 | * Important parameters: 28 | * numConnections: Number of connections to one host. 29 | * socketTimeoutMillis: Socket timeout when trying to establish the new connection. 30 | * maxWaitMillis: If the connection pool is full, how long to wait. 31 | */ 32 | public class JedisClientHelper implements RedisClientHelper { 33 | 34 | public JedisPool createClient(RedisClientConfig redisClientConfig, EndPoint endPoint) { 35 | JedisPoolConfig config = new JedisPoolConfig(); 36 | config.setMaxWait(redisClientConfig.getMaxWaitMillis()); 37 | config.setMaxActive(redisClientConfig.getNumConnections()); 38 | config.setMaxIdle(redisClientConfig.getNumConnections()); 39 | // Deal with idle connection eviction. 40 | config.setTestOnBorrow(false); 41 | config.setTestOnReturn(false); 42 | config.setTestWhileIdle(true); 43 | config.setMinEvictableIdleTimeMillis(5 * 60 * 1000); 44 | config.setTimeBetweenEvictionRunsMillis(3 * 60 * 1000); 45 | config.setNumTestsPerEvictionRun(redisClientConfig.getNumConnections()); 46 | JedisPool jedisPool = new JedisPool( 47 | config, endPoint.host, endPoint.port, redisClientConfig.getSocketTimeoutMillis()); 48 | return jedisPool; 49 | } 50 | 51 | public void destroyClient(JedisPool jedisPool) { 52 | jedisPool.destroy(); 53 | } 54 | 55 | public boolean clientPing(JedisPool jedisPool) { 56 | boolean result = false; 57 | try { 58 | result = RedisUtils.executeWithConnection( 59 | jedisPool, 60 | new Function() { 61 | @Override 62 | public String apply(Jedis conn) { 63 | return conn.ping(); 64 | } 65 | } 66 | ).equals("PONG"); 67 | } catch (Exception e) { 68 | // failed ping 69 | } 70 | return result; 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/commons/ostrich/OstrichAdminService.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.commons.ostrich; 18 | 19 | import com.google.common.collect.Maps; 20 | import com.twitter.ostrich.admin.AdminHttpService; 21 | import com.twitter.ostrich.admin.AdminServiceFactory; 22 | import com.twitter.ostrich.admin.CustomHttpHandler; 23 | import com.twitter.ostrich.admin.RuntimeEnvironment; 24 | import com.twitter.ostrich.admin.StatsFactory; 25 | import com.twitter.util.Duration; 26 | import org.slf4j.Logger; 27 | import org.slf4j.LoggerFactory; 28 | import scala.Option; 29 | import scala.collection.Map$; 30 | import scala.collection.immutable.List; 31 | import scala.collection.immutable.List$; 32 | import scala.util.matching.Regex; 33 | 34 | import java.util.Map; 35 | import java.util.Properties; 36 | import java.util.concurrent.TimeUnit; 37 | 38 | public class OstrichAdminService { 39 | 40 | private static final Logger LOG = LoggerFactory.getLogger(OstrichAdminService.class); 41 | private final int mPort; 42 | private final Map mCustomHttpHandlerMap = Maps.newHashMap(); 43 | 44 | public OstrichAdminService(int port) { 45 | this.mPort = port; 46 | } 47 | 48 | public void addHandler(String path, CustomHttpHandler handler) { 49 | this.mCustomHttpHandlerMap.put(path, handler); 50 | } 51 | 52 | public void start() { 53 | Duration[] defaultLatchIntervals = {Duration.apply(1, TimeUnit.MINUTES)}; 54 | @SuppressWarnings("deprecation") 55 | AdminServiceFactory adminServiceFactory = new AdminServiceFactory( 56 | this.mPort, 57 | 20, 58 | List$.MODULE$.empty(), 59 | Option.empty(), 60 | List$.MODULE$.empty(), 61 | Map$.MODULE$.empty(), 62 | List.fromArray(defaultLatchIntervals)); 63 | RuntimeEnvironment runtimeEnvironment = new RuntimeEnvironment(this); 64 | AdminHttpService service = adminServiceFactory.apply(runtimeEnvironment); 65 | for (Map.Entry entry : this.mCustomHttpHandlerMap.entrySet()) { 66 | service.httpServer().createContext(entry.getKey(), entry.getValue()); 67 | } 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/redis/RedisUtils.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.redis; 18 | 19 | import com.google.common.base.Preconditions; 20 | import com.twitter.util.Function; 21 | import org.slf4j.Logger; 22 | import org.slf4j.LoggerFactory; 23 | import redis.clients.jedis.Jedis; 24 | import redis.clients.jedis.JedisPool; 25 | import redis.clients.jedis.exceptions.JedisConnectionException; 26 | 27 | public class RedisUtils { 28 | 29 | private static final Logger LOG = LoggerFactory.getLogger(RedisUtils.class); 30 | 31 | /** 32 | * Gets the connection from the connection pool and adds the wrapper catch/finally block for the 33 | * given function. 34 | * 35 | * This helper method saves the trouble of dealing with redis connection. When we got 36 | * JedisConnectionException, we will discard this connection. Otherwise, we return the connection 37 | * to the connection pool. 38 | * 39 | * @param jedisPool Jedis connection pool 40 | * @param redisDBNum Redis DB number (index) (if redisDBNum == -1, don't select a DB ) 41 | * @param func The function to execute inside the catch/finally block. 42 | * @return A Resp object, which is the return value of wrapped function. 43 | */ 44 | public static Resp executeWithConnection(JedisPool jedisPool, 45 | int redisDBNum, 46 | Function func) { 47 | Preconditions.checkNotNull(jedisPool); 48 | Preconditions.checkNotNull(func); 49 | Jedis conn = null; 50 | boolean gotJedisConnException = false; 51 | try { 52 | conn = jedisPool.getResource(); 53 | selectRedisDB(conn, redisDBNum); 54 | return func.apply(conn); 55 | } catch (JedisConnectionException e) { 56 | jedisPool.returnBrokenResource(conn); 57 | gotJedisConnException = true; 58 | throw e; 59 | } finally { 60 | if (conn != null && !gotJedisConnException) { 61 | jedisPool.returnResource(conn); 62 | } 63 | } 64 | } 65 | 66 | /** 67 | * Same as above, but without the Redis DB index selection. 68 | * 69 | */ 70 | public static Resp executeWithConnection(JedisPool jedisPool, Function func) 71 | throws Exception { 72 | // NOTE: use the default DB number -1 here to indicate no DB will be selected. 73 | return executeWithConnection(jedisPool, -1, func); 74 | } 75 | 76 | /** 77 | * Select the Redis DB number (index) for the given Jedis connection. 78 | * If the DB number == -1, don't do anything. 79 | * 80 | * @param conn Jedis connection 81 | * @param redisDBNum Redis DB number (index) 82 | */ 83 | private static void selectRedisDB(Jedis conn, int redisDBNum) { 84 | if (redisDBNum != -1) { 85 | conn.select(redisDBNum); 86 | } 87 | } 88 | } 89 | -------------------------------------------------------------------------------- /src/test/java/com/pinterest/pinlater/PinLaterQueueConfigTest.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater; 18 | 19 | import org.junit.Assert; 20 | import org.junit.Test; 21 | import org.junit.runner.RunWith; 22 | import org.junit.runners.JUnit4; 23 | 24 | import java.io.IOException; 25 | 26 | @RunWith(JUnit4.class) 27 | 28 | public class PinLaterQueueConfigTest { 29 | 30 | private static final String TEST_QUEUE_CONFIG = 31 | "{\n" + 32 | " \"queues\": [\n" + 33 | " {\n" + 34 | " \"name\": \"test_queue_1\",\n" + 35 | " \"queueConfig\": {\n" + 36 | " \"maxJobsPerSecond\": 0\n" + 37 | " }\n" + 38 | " },\n" + 39 | " {\n" + 40 | " \"name\": \"test_queue_2\",\n" + 41 | " \"queueConfig\": {\n" + 42 | " \"maxJobsPerSecond\": 100\n" + 43 | " }\n" + 44 | " }\n" + 45 | " ]\n" + 46 | "}"; 47 | 48 | private final PinLaterQueueConfig queueConfig = new PinLaterQueueConfig("", "", false); 49 | 50 | @Test 51 | public void testRateLimit() throws IOException { 52 | queueConfig.getQueueConfigSchemaRef().set( 53 | PinLaterQueueConfig.QueueConfigSchema.load(TEST_QUEUE_CONFIG.getBytes())); 54 | 55 | queueConfig.getNumPinLaterServers().set(10); 56 | queueConfig.rebuild(); 57 | 58 | Assert.assertEquals(Double.MAX_VALUE, queueConfig.getDequeueRate("queue_not_in_config"), 0.01); 59 | Assert.assertTrue(queueConfig.allowDequeue("queue_not_in_config", 1)); 60 | Assert.assertTrue(queueConfig.allowDequeue("queue_not_in_config", 10)); 61 | Assert.assertTrue(queueConfig.allowDequeue("queue_not_in_config", 100)); 62 | Assert.assertTrue(queueConfig.allowDequeue("queue_not_in_config", Integer.MAX_VALUE)); 63 | 64 | Assert.assertEquals(0.0, queueConfig.getDequeueRate("test_queue_1"), 0.01); 65 | Assert.assertFalse(queueConfig.allowDequeue("test_queue_1", 1)); 66 | Assert.assertFalse(queueConfig.allowDequeue("test_queue_1", 10)); 67 | Assert.assertFalse(queueConfig.allowDequeue("test_queue_1", 100)); 68 | Assert.assertFalse(queueConfig.allowDequeue("test_queue_1", Integer.MAX_VALUE)); 69 | 70 | Assert.assertEquals(10.0, queueConfig.getDequeueRate("test_queue_2"), 0.01); 71 | Assert.assertTrue(queueConfig.allowDequeue("test_queue_2", 10)); 72 | Assert.assertFalse(queueConfig.allowDequeue("test_queue_2", 1)); 73 | 74 | queueConfig.getNumPinLaterServers().set(1); 75 | queueConfig.rebuild(); 76 | 77 | Assert.assertEquals(Double.MAX_VALUE, queueConfig.getDequeueRate("queue_not_in_config"), 0.01); 78 | Assert.assertEquals(0.0, queueConfig.getDequeueRate("test_queue_1"), 0.01); 79 | Assert.assertEquals(100.0, queueConfig.getDequeueRate("test_queue_2"), 0.01); 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/client/PinLaterClient.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.client; 18 | 19 | import com.pinterest.pinlater.thrift.PinLater; 20 | 21 | import com.twitter.common.zookeeper.ServerSet; 22 | import com.twitter.finagle.Service; 23 | import com.twitter.finagle.builder.ClientBuilder; 24 | import com.twitter.finagle.thrift.ClientId; 25 | import com.twitter.finagle.thrift.ThriftClientFramedCodec; 26 | import com.twitter.finagle.thrift.ThriftClientRequest; 27 | import com.twitter.finagle.zookeeper.ZookeeperServerSetCluster; 28 | import com.twitter.util.Duration; 29 | import org.apache.commons.cli.CommandLine; 30 | import org.apache.thrift.protocol.TBinaryProtocol; 31 | import scala.Option; 32 | 33 | import java.net.InetSocketAddress; 34 | import java.util.concurrent.TimeUnit; 35 | 36 | /** 37 | * Class to hold the service and iface for a PinLater client. 38 | */ 39 | public class PinLaterClient { 40 | 41 | private final Service service; 42 | private final PinLater.ServiceIface iface; 43 | 44 | /** 45 | * Returns instance of PinLaterClient that creates connection that is not stored in the client 46 | * connection pool. 47 | * @param cmdLine 48 | */ 49 | public PinLaterClient(CommandLine cmdLine) { 50 | this( 51 | cmdLine.getOptionValue("host"), 52 | Integer.parseInt(cmdLine.getOptionValue("port")), 53 | Integer.parseInt(cmdLine.getOptionValue("concurrency", "1"))); 54 | } 55 | 56 | public PinLaterClient(String host, int port, int concurrency) { 57 | this.service = ClientBuilder.safeBuild( 58 | ClientBuilder.get() 59 | .hosts(new InetSocketAddress(host, port)) 60 | .codec(ThriftClientFramedCodec.apply(Option.apply(new ClientId("pinlaterclient")))) 61 | .hostConnectionLimit(concurrency) 62 | .tcpConnectTimeout(Duration.apply(2, TimeUnit.SECONDS)) 63 | .requestTimeout(Duration.apply(10, TimeUnit.SECONDS)) 64 | .retries(1)); 65 | this.iface = new PinLater.ServiceToClient(service, new TBinaryProtocol.Factory()); 66 | } 67 | 68 | public PinLaterClient(ServerSet serverSet, int concurrency) { 69 | ZookeeperServerSetCluster cluster = new ZookeeperServerSetCluster(serverSet); 70 | ClientBuilder builder = ClientBuilder.get().cluster(cluster); 71 | this.service = ClientBuilder.safeBuild( 72 | builder.codec(ThriftClientFramedCodec.get()) 73 | .tcpConnectTimeout(Duration.apply(2, TimeUnit.SECONDS)) 74 | .requestTimeout(Duration.apply(10, TimeUnit.SECONDS)) 75 | .hostConnectionLimit(concurrency)); 76 | this.iface = new PinLater.ServiceToClient(service, new TBinaryProtocol.Factory()); 77 | } 78 | 79 | public Service getService() { 80 | return service; 81 | } 82 | 83 | public PinLater.ServiceIface getIface() { 84 | return iface; 85 | } 86 | } 87 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/client/PinLaterClientTool.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.client; 18 | 19 | import org.apache.commons.cli.CommandLine; 20 | import org.apache.commons.cli.CommandLineParser; 21 | import org.apache.commons.cli.OptionBuilder; 22 | import org.apache.commons.cli.Options; 23 | import org.apache.commons.cli.PosixParser; 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | /** 28 | * Client tool to talk to PinLater. This can be used for correctness as well as 29 | * load/performance testing. 30 | */ 31 | public class PinLaterClientTool { 32 | 33 | private static final Logger LOG = LoggerFactory.getLogger(PinLaterClientTool.class); 34 | 35 | private static Options getOptions() { 36 | Options options = new Options(); 37 | options.addOption(OptionBuilder.withLongOpt("host").isRequired().hasArg() 38 | .withDescription("PinLater server hostname").create()); 39 | options.addOption(OptionBuilder.withLongOpt("port").isRequired().hasArg() 40 | .withType(Number.class).withDescription("PinLater server port").create()); 41 | 42 | options.addOption(OptionBuilder.withLongOpt("mode").isRequired().hasArg() 43 | .withDescription("Mode: create, enqueue, dequeue, check_dequeue, lookup, " 44 | + "get_job_count, get_queue_names").create()); 45 | options.addOption(OptionBuilder.withLongOpt("queue").hasArg() 46 | .withDescription("Queue name").create()); 47 | options.addOption(OptionBuilder.withLongOpt("concurrency").hasArg() 48 | .withDescription("Query issue concurrency").create()); 49 | options.addOption(OptionBuilder.withLongOpt("batch_size").hasArg() 50 | .withDescription("Query batch size").create()); 51 | options.addOption(OptionBuilder.withLongOpt("num_queries").hasArg() 52 | .withDescription("How many enqueue/dequeue queries to issue, -1 for unlimited").create()); 53 | options.addOption(OptionBuilder.withLongOpt("dequeue_success_percent").hasArg() 54 | .withDescription("Dequeue success percent to use").create()); 55 | options.addOption(OptionBuilder.withLongOpt("job_descriptor").hasArg() 56 | .withDescription("Job descriptor to lookup").create()); 57 | options.addOption(OptionBuilder.withLongOpt("priority").hasArg() 58 | .withDescription("Job priority for enqueue").create()); 59 | options.addOption(OptionBuilder.withLongOpt("job_state").hasArg() 60 | .withDescription("Job state to search for").create()); 61 | options.addOption(OptionBuilder.withLongOpt("count_future_jobs").hasArg() 62 | .withDescription("Count future jobs instead of current jobs").create()); 63 | 64 | return options; 65 | } 66 | 67 | public static void main(String[] args) { 68 | try { 69 | CommandLineParser parser = new PosixParser(); 70 | CommandLine cmdLine = parser.parse(getOptions(), args); 71 | new PinLaterQueryIssuer(cmdLine).run(); 72 | } catch (Exception e) { 73 | LOG.error("Exception in client tool", e); 74 | System.exit(1); 75 | } 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/test/java/com/pinterest/pinlater/backends/mysql/MySQLQueueMonitorTest.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.mysql; 18 | 19 | import com.pinterest.pinlater.backends.common.BackendQueueMonitorBaseTest; 20 | 21 | import com.google.common.collect.ImmutableMap; 22 | import org.apache.commons.configuration.ConfigurationException; 23 | import org.apache.commons.configuration.PropertiesConfiguration; 24 | import org.junit.After; 25 | import org.junit.Assume; 26 | import org.junit.Before; 27 | 28 | import java.io.InputStream; 29 | 30 | public class MySQLQueueMonitorTest extends BackendQueueMonitorBaseTest { 32 | 33 | private static final String QUEUE_NAME = "mysql_queue_monitor_test"; 34 | 35 | private PinLaterMySQLBackend backend; 36 | private PropertiesConfiguration configuration; 37 | 38 | @Override 39 | protected String getQueueName() { 40 | return QUEUE_NAME; 41 | } 42 | 43 | @Override 44 | protected PinLaterMySQLBackend getBackend() { 45 | return backend; 46 | } 47 | 48 | @Override 49 | protected MySQLQueueMonitor createQueueMonitor(long jobClaimedTimeoutMillis, 50 | long jobSucceededGCTimeoutMillis, 51 | long jobFailedGCTimeoutMillis) { 52 | InputStream mysqlConfigStream = ClassLoader.getSystemResourceAsStream("mysql.local.json"); 53 | ImmutableMap shardMap = 54 | MySQLBackendUtils.buildShardMap(mysqlConfigStream, configuration); 55 | return new MySQLQueueMonitor(shardMap, 56 | 1000, // update max size 57 | 3, // auto retries 58 | 1, // log interval 59 | jobClaimedTimeoutMillis, 60 | jobSucceededGCTimeoutMillis, 61 | jobFailedGCTimeoutMillis, 62 | 3 // max queue priority 63 | ); 64 | } 65 | 66 | @Before 67 | public void beforeTest() throws Exception { 68 | // If there is no local MySQL, skip this test. 69 | boolean isLocalMysqlRunning = LocalMySQLChecker.isRunning(); 70 | Assume.assumeTrue(isLocalMysqlRunning); 71 | 72 | configuration = new PropertiesConfiguration(); 73 | try { 74 | configuration.load(ClassLoader.getSystemResourceAsStream("pinlater.test.properties")); 75 | } catch (ConfigurationException e) { 76 | throw new RuntimeException(e); 77 | } 78 | System.setProperty("backend_config", "mysql.local.json"); 79 | 80 | backend = new PinLaterMySQLBackend( 81 | configuration, "localhost", System.currentTimeMillis()); 82 | 83 | // Delete the test queue, if it exists already, for some reason. 84 | backend.deleteQueue(QUEUE_NAME).get(); 85 | 86 | // Now create it. 87 | backend.createQueue(QUEUE_NAME).get(); 88 | } 89 | 90 | @After 91 | public void afterTest() { 92 | boolean isLocalMysqlRunning = LocalMySQLChecker.isRunning(); 93 | if (!isLocalMysqlRunning) { 94 | return; 95 | } 96 | 97 | // Clean up the test queue. 98 | backend.deleteQueue(QUEUE_NAME).get(); 99 | } 100 | } 101 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/redis/RedisClientConfig.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.redis; 18 | 19 | public class RedisClientConfig { 20 | 21 | private String serviceName; 22 | private int numConnections; 23 | private int maxWaitMillis; 24 | 25 | /** Timeout to setup an actual connection to Redis */ 26 | private int socketTimeoutMillis; 27 | 28 | /** Timeout for a request including any replies */ 29 | private int totalRequestTimeoutMillis; 30 | 31 | /** Max number of retries per request */ 32 | private int maxRetries; 33 | 34 | private int redisHealthCheckConsecutiveFailures; 35 | private int redisHealthCheckConsecutiveSuccesses; 36 | private int redisHealthCheckPingIntervalSeconds; 37 | 38 | public RedisClientConfig() { 39 | this.serviceName = "redisservice"; 40 | this.numConnections = 1000; 41 | this.maxWaitMillis = 250; 42 | this.socketTimeoutMillis = 1000; 43 | this.totalRequestTimeoutMillis = 1000; 44 | this.maxRetries = 3; 45 | this.redisHealthCheckConsecutiveFailures = 6; 46 | this.redisHealthCheckConsecutiveSuccesses = 6; 47 | this.redisHealthCheckPingIntervalSeconds = 5; 48 | } 49 | 50 | public String getServiceName() { 51 | return this.serviceName; 52 | } 53 | 54 | public void setServiceName(String serviceName) { 55 | this.serviceName = serviceName; 56 | } 57 | 58 | public int getNumConnections() { 59 | return this.numConnections; 60 | } 61 | 62 | public void setNumConnections(int numConnections) { 63 | this.numConnections = numConnections; 64 | } 65 | 66 | public int getMaxWaitMillis() { 67 | return this.maxWaitMillis; 68 | } 69 | 70 | public void setMaxWaitMillis(int maxWaitMillis) { 71 | this.maxWaitMillis = maxWaitMillis; 72 | } 73 | 74 | public int getSocketTimeoutMillis() { 75 | return this.socketTimeoutMillis; 76 | } 77 | 78 | public void setSocketTimeoutMillis(int socketTimeoutMillis) { 79 | this.socketTimeoutMillis = socketTimeoutMillis; 80 | } 81 | 82 | public int getRedisHealthCheckConsecutiveFailures() { 83 | return this.redisHealthCheckConsecutiveFailures; 84 | } 85 | 86 | public void setRedisHealthCheckConsecutiveFailures(int redisHealthCheckConsecutiveFailures) { 87 | this.redisHealthCheckConsecutiveFailures = redisHealthCheckConsecutiveFailures; 88 | } 89 | 90 | public int getRedisHealthCheckConsecutiveSuccesses() { 91 | return this.redisHealthCheckConsecutiveSuccesses; 92 | } 93 | 94 | public void setRedisHealthCheckConsecutiveSuccesses(int redisHealthCheckConsecutiveSuccesses) { 95 | this.redisHealthCheckConsecutiveSuccesses = redisHealthCheckConsecutiveSuccesses; 96 | } 97 | 98 | public int getRedisHealthCheckPingIntervalSeconds() { 99 | return this.redisHealthCheckPingIntervalSeconds; 100 | } 101 | 102 | public void setRedisHealthCheckPingIntervalSeconds(int redisHealthCheckPingIntervalSeconds) { 103 | this.redisHealthCheckPingIntervalSeconds = redisHealthCheckPingIntervalSeconds; 104 | } 105 | 106 | public int getTotalRequestTimeoutMillis() { 107 | return this.totalRequestTimeoutMillis; 108 | } 109 | 110 | public void setTotalRequestTimeoutMillis(int totalRequestTimeoutMillis) { 111 | this.totalRequestTimeoutMillis = totalRequestTimeoutMillis; 112 | } 113 | 114 | public int getMaxRetries() { 115 | return this.maxRetries; 116 | } 117 | 118 | public void setMaxRetries(int maxRetries) { 119 | this.maxRetries = maxRetries; 120 | } 121 | } 122 | -------------------------------------------------------------------------------- /src/test/java/com/pinterest/pinlater/backends/mysql/MySQLHealthMonitorTest.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.mysql; 18 | 19 | import com.google.common.collect.Maps; 20 | import com.google.common.collect.Sets; 21 | import org.junit.Assert; 22 | import org.junit.Test; 23 | 24 | import java.util.Map; 25 | 26 | public class MySQLHealthMonitorTest { 27 | 28 | @Test 29 | public void testBasicFunctionality() throws Exception { 30 | MySQLHealthMonitor healthMonitor = 31 | new MySQLHealthMonitor(Sets.newHashSet("1", "2", "3"), 10, 7, 100); 32 | Assert.assertTrue(healthMonitor.isHealthy("1")); 33 | 34 | // Till we reach 10 samples, isHealthy should be true. 35 | for (int i = 0; i < 9; i++) { 36 | healthMonitor.recordSample("1", false); 37 | } 38 | Assert.assertTrue(healthMonitor.isHealthy("1")); 39 | 40 | // One more sample should take shard 1 to unhealthy state. 41 | healthMonitor.recordSample("1", true); 42 | Assert.assertFalse(healthMonitor.isHealthy("1")); 43 | 44 | // Other two shards are still healthy though. 45 | Assert.assertTrue(healthMonitor.isHealthy("2")); 46 | Assert.assertTrue(healthMonitor.isHealthy("3")); 47 | 48 | // Shard 1 health should be restored after 100ms. 49 | Thread.sleep(100); 50 | Assert.assertTrue(healthMonitor.isHealthy("1")); 51 | 52 | // Add 4 failures + 6 successes to shard 2, which should make it unhealthy. 53 | for (int i = 0; i < 10; i++) { 54 | healthMonitor.recordSample("2", i >= 4); 55 | } 56 | Assert.assertFalse(healthMonitor.isHealthy("2")); 57 | 58 | // Shard 2 health should be restored after 100ms. 59 | Thread.sleep(100); 60 | Assert.assertTrue(healthMonitor.isHealthy("2")); 61 | 62 | // Repeating above test should yield same result now. 63 | for (int i = 0; i < 10; i++) { 64 | healthMonitor.recordSample("2", i >= 4); 65 | } 66 | Assert.assertFalse(healthMonitor.isHealthy("2")); 67 | } 68 | 69 | @Test 70 | public void testNonExistentShardId() { 71 | MySQLHealthMonitor healthMonitor = 72 | new MySQLHealthMonitor(Sets.newHashSet("1", "2", "3"), 10, 7, 100); 73 | Assert.assertFalse(healthMonitor.isHealthy("5")); 74 | } 75 | 76 | @Test 77 | public void testUpdateHealthMap() { 78 | MySQLHealthMonitor healthMonitor = 79 | new MySQLHealthMonitor(Sets.newHashSet("1", "2", "3"), 10, 7, 100); 80 | 81 | for (int i = 0; i < 10; i++) { 82 | healthMonitor.recordSample("1", false); 83 | healthMonitor.recordSample("2", false); 84 | } 85 | 86 | // Start with three shards, 1 & 2 are unhealthy and 3 is healthy 87 | Assert.assertFalse(healthMonitor.isHealthy("1")); 88 | Assert.assertFalse(healthMonitor.isHealthy("2")); 89 | Assert.assertTrue(healthMonitor.isHealthy("3")); 90 | 91 | Map updateMap = Maps.newHashMap(); 92 | 93 | // No change to shard 1, remove shard 3 and create two new shards (reuse shard ID 2) 94 | updateMap.put("1", false); 95 | updateMap.put("2", true); 96 | updateMap.put("4", true); 97 | 98 | healthMonitor.updateHealthMap(updateMap); 99 | 100 | // No change to shard 1, should remain unhealthy 101 | Assert.assertFalse(healthMonitor.isHealthy("1")); 102 | 103 | // New shards should be healthy 104 | Assert.assertTrue(healthMonitor.isHealthy("2")); 105 | Assert.assertTrue(healthMonitor.isHealthy("4")); 106 | 107 | // Checking non-existing shard should return false 108 | Assert.assertFalse(healthMonitor.isHealthy("3")); 109 | } 110 | } 111 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/redis/RedisPools.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.redis; 18 | 19 | import org.apache.commons.configuration.PropertiesConfiguration; 20 | import org.slf4j.Logger; 21 | import org.slf4j.LoggerFactory; 22 | import redis.clients.jedis.Jedis; 23 | import redis.clients.jedis.JedisPool; 24 | import redis.clients.jedis.JedisPoolConfig; 25 | import redis.clients.jedis.exceptions.JedisConnectionException; 26 | 27 | import java.util.concurrent.TimeUnit; 28 | 29 | /** 30 | * Helper class used by the PinLaterRedisBackend to encapsulate Redis pools. 31 | */ 32 | public class RedisPools { 33 | 34 | private static final Logger LOG = LoggerFactory.getLogger(RedisPools.class); 35 | 36 | private final String host; 37 | private final int port; 38 | private final boolean dequeueOnly; 39 | private final JedisPool generalRedisPool; 40 | private final JedisPool monitorRedisPool; 41 | 42 | public RedisPools(PropertiesConfiguration configuration, String host, int port, 43 | boolean dequeueOnly) { 44 | int numGeneralConnections = configuration.getInt("BACKEND_CONNECTIONS_PER_SHARD"); 45 | int generalSocketTimeoutMillis = (int) TimeUnit.SECONDS.toMillis( 46 | configuration.getInt("BACKEND_SOCKET_TIMEOUT_SECONDS")); 47 | int numMonitorConnections = configuration.getInt("MONITOR_CONNECTIONS_PER_SHARD", 3); 48 | int monitorSocketTimeoutMillis = (int) TimeUnit.SECONDS.toMillis( 49 | configuration.getInt("MONITOR_SOCKET_TIMEOUT_SECONDS", 10)); 50 | int maxWaitMillis = configuration.getInt("BACKEND_CONNECTION_MAX_WAIT_MILLIS"); 51 | this.host = host; 52 | this.port = port; 53 | this.dequeueOnly = dequeueOnly; 54 | this.generalRedisPool = createRedisPool( 55 | host, port, numGeneralConnections, 56 | maxWaitMillis, generalSocketTimeoutMillis); 57 | this.monitorRedisPool = createRedisPool( 58 | host, port, numMonitorConnections, 59 | maxWaitMillis, monitorSocketTimeoutMillis); 60 | } 61 | 62 | public String getHost() { return host; } 63 | 64 | public int getPort() { return port; } 65 | 66 | public boolean getDequeueOnly() { return dequeueOnly; } 67 | 68 | public JedisPool getGeneralRedisPool() { return generalRedisPool; } 69 | 70 | public JedisPool getMonitorRedisPool() { return monitorRedisPool; } 71 | 72 | private static JedisPool createRedisPool( 73 | String host, int port, int poolSize, int maxWaitMillis, int socketTimeoutMillis) { 74 | JedisPoolConfig config = new JedisPoolConfig(); 75 | config.setMaxWait(maxWaitMillis); 76 | config.setMaxActive(poolSize); 77 | config.setMaxIdle(poolSize); 78 | // Deal with idle connection eviction. 79 | config.setTestOnBorrow(false); 80 | config.setTestOnReturn(false); 81 | config.setTestWhileIdle(true); 82 | config.setMinEvictableIdleTimeMillis(5 * 60 * 1000); 83 | config.setTimeBetweenEvictionRunsMillis(3 * 60 * 1000); 84 | config.setNumTestsPerEvictionRun(poolSize); 85 | 86 | JedisPool pool = new JedisPool(config, host, port, socketTimeoutMillis); 87 | // Force connection pool initialization. 88 | Jedis jedis = null; 89 | try { 90 | jedis = pool.getResource(); 91 | } catch (JedisConnectionException e) { 92 | LOG.error( 93 | String.format("Failed to get a redis connection when creating redis pool, " 94 | + "host: %s, port: %d", host, port), 95 | e); 96 | } finally { 97 | pool.returnResource(jedis); 98 | } 99 | return pool; 100 | } 101 | } 102 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/PinLaterServer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater; 18 | 19 | import com.pinterest.pinlater.backends.mysql.PinLaterMySQLBackend; 20 | import com.pinterest.pinlater.backends.redis.PinLaterRedisBackend; 21 | import com.pinterest.pinlater.commons.ostrich.OstrichAdminService; 22 | import com.pinterest.pinlater.commons.serviceframework.ServiceShutdownHook; 23 | import com.pinterest.pinlater.thrift.PinLater; 24 | 25 | import com.twitter.finagle.builder.ServerBuilder; 26 | import com.twitter.finagle.stats.OstrichStatsReceiver; 27 | import com.twitter.finagle.thrift.ThriftServerFramedCodec; 28 | import com.twitter.ostrich.stats.Stats; 29 | import com.twitter.util.Duration; 30 | import org.apache.commons.configuration.ConfigurationException; 31 | import org.apache.commons.configuration.PropertiesConfiguration; 32 | import org.apache.thrift.protocol.TBinaryProtocol; 33 | import org.slf4j.Logger; 34 | import org.slf4j.LoggerFactory; 35 | 36 | import java.io.InputStream; 37 | import java.net.InetAddress; 38 | import java.net.InetSocketAddress; 39 | import java.util.concurrent.TimeUnit; 40 | 41 | /** 42 | * Main class for PinLaterServer. 43 | */ 44 | public class PinLaterServer { 45 | 46 | private static final Logger LOG = LoggerFactory.getLogger(PinLaterServer.class); 47 | private static final PropertiesConfiguration CONFIGURATION = new PropertiesConfiguration(); 48 | private static final long SERVER_START_TIME_MILLIS = System.currentTimeMillis(); 49 | 50 | static { 51 | try { 52 | CONFIGURATION.load(ClassLoader.getSystemResourceAsStream( 53 | System.getProperty("server_config"))); 54 | } catch (ConfigurationException e) { 55 | LOG.error("Failed to load server configuration", e); 56 | throw new RuntimeException(e); 57 | } 58 | } 59 | 60 | private static PinLaterBackendIface getBackendIface(String backend, String serverHostName) 61 | throws Exception { 62 | InputStream backendConfigStream = ClassLoader.getSystemResourceAsStream( 63 | System.getProperty("backend_config")); 64 | if (backend != null && backend.equals("redis")) { 65 | return new PinLaterRedisBackend( 66 | CONFIGURATION, backendConfigStream, serverHostName, SERVER_START_TIME_MILLIS); 67 | } else { 68 | return new PinLaterMySQLBackend( 69 | CONFIGURATION, serverHostName, SERVER_START_TIME_MILLIS); 70 | } 71 | } 72 | 73 | public static void main(String[] args) { 74 | try { 75 | String serverHostName = InetAddress.getLocalHost().getHostName(); 76 | PinLaterQueueConfig queueConfig = new PinLaterQueueConfig(CONFIGURATION); 77 | queueConfig.initialize(); 78 | String backend = CONFIGURATION.getString("PINLATER_BACKEND"); 79 | PinLaterBackendIface backendIFace = getBackendIface(backend, serverHostName); 80 | PinLaterServiceImpl serviceImpl = new PinLaterServiceImpl(backendIFace, queueConfig); 81 | PinLater.Service service = new PinLater.Service(serviceImpl, new TBinaryProtocol.Factory()); 82 | ServiceShutdownHook.register(ServerBuilder.safeBuild( 83 | service, 84 | ServerBuilder.get() 85 | .name("PinLaterService") 86 | .codec(ThriftServerFramedCodec.get()) 87 | .hostConnectionMaxIdleTime(Duration.fromTimeUnit( 88 | CONFIGURATION.getInt("SERVER_CONN_MAX_IDLE_TIME_MINUTES"), TimeUnit.MINUTES)) 89 | .maxConcurrentRequests(CONFIGURATION.getInt("MAX_CONCURRENT_REQUESTS")) 90 | .reportTo(new OstrichStatsReceiver(Stats.get(""))) 91 | .bindTo(new InetSocketAddress(CONFIGURATION.getInt("THRIFT_PORT"))))); 92 | new OstrichAdminService(CONFIGURATION.getInt("OSTRICH_PORT")).start(); 93 | 94 | LOG.info("\n#######################################" 95 | + "\n# Ready To Serve Requests. #" 96 | + "\n#######################################"); 97 | } catch (Exception e) { 98 | LOG.error("Failed to start the pinlater server", e); 99 | System.exit(1); 100 | } 101 | } 102 | } 103 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/common/BackendQueueMonitorBase.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.common; 18 | 19 | import com.google.common.annotations.VisibleForTesting; 20 | import com.google.common.base.Preconditions; 21 | import com.google.common.collect.ImmutableMap; 22 | import org.apache.commons.configuration.PropertiesConfiguration; 23 | 24 | import java.util.Map; 25 | import java.util.concurrent.TimeUnit; 26 | 27 | /** 28 | * The base class of backend queue monitor. It implements a scheduled task used by the 29 | * PinLaterBackendBase for various types of job queue cleanup, including ACK timeouts and GC'ing 30 | * finished jobs. The detailed logic of the cleanup operations specific to each backend should be 31 | * implemented in the subclass. T is the type of datastore pool used by PinLaterBackendBase. 32 | */ 33 | public abstract class BackendQueueMonitorBase implements Runnable { 34 | 35 | private final ImmutableMap shardMap; 36 | private final int updateMaxSize; 37 | private final int maxAutoRetries; 38 | private final int logInterval; 39 | private final long jobClaimedTimeoutMillis; 40 | private final long jobSucceededGCTimeoutMillis; 41 | private final long jobFailedGCTimeoutMillis; 42 | protected final int numPriorityLevels; 43 | 44 | protected ImmutableMap getShardMap() { 45 | return shardMap; 46 | } 47 | 48 | protected final int getUpdateMaxSize() { 49 | return updateMaxSize; 50 | } 51 | 52 | protected final int getMaxAutoRetries() { 53 | return maxAutoRetries; 54 | } 55 | 56 | public int getLogInterval() { 57 | return logInterval; 58 | } 59 | 60 | protected final long getJobClaimedTimeoutMillis() { 61 | return jobClaimedTimeoutMillis; 62 | } 63 | 64 | protected final long getJobSucceededGCTimeoutMillis() { 65 | return jobSucceededGCTimeoutMillis; 66 | } 67 | 68 | protected final long getJobFailedGCTimeoutMillis() { 69 | return jobFailedGCTimeoutMillis; 70 | } 71 | 72 | public BackendQueueMonitorBase(ImmutableMap shardMap, 73 | PropertiesConfiguration configuration) { 74 | this(shardMap, 75 | configuration.getInt("BACKEND_MONITOR_UPDATE_MAX_SIZE"), 76 | configuration.getInt("BACKEND_NUM_AUTO_RETRIES"), 77 | configuration.getInt("MONITOR_LOG_INTERVAL", 1), 78 | TimeUnit.SECONDS.toMillis( 79 | configuration.getInt("BACKEND_MONITOR_JOB_CLAIMED_TIMEOUT_SECONDS")), 80 | TimeUnit.HOURS.toMillis(configuration.getInt( 81 | "BACKEND_MONITOR_JOB_SUCCEEDED_GC_TTL_HOURS")), 82 | TimeUnit.HOURS.toMillis(configuration.getInt( 83 | "BACKEND_MONITOR_JOB_FAILED_GC_TTL_HOURS")), 84 | configuration.getInt("NUM_PRIORITY_LEVELS")); 85 | } 86 | 87 | @VisibleForTesting 88 | public BackendQueueMonitorBase(ImmutableMap shardMap, 89 | int updateMaxSize, int maxAutoRetries, int logInterval, 90 | long jobClaimedTimeoutMillis, long jobSucceededGCTimeoutMillis, 91 | long jobFailedGCTimeoutMillis, 92 | int numPriorityLevels) { 93 | this.shardMap = Preconditions.checkNotNull(shardMap); 94 | this.updateMaxSize = updateMaxSize; 95 | this.maxAutoRetries = maxAutoRetries; 96 | this.logInterval = logInterval; 97 | this.jobClaimedTimeoutMillis = jobClaimedTimeoutMillis; 98 | this.jobSucceededGCTimeoutMillis = jobSucceededGCTimeoutMillis; 99 | this.jobFailedGCTimeoutMillis = jobFailedGCTimeoutMillis; 100 | this.numPriorityLevels = numPriorityLevels; 101 | } 102 | 103 | @Override 104 | public void run() { 105 | final long runStartMillis = System.currentTimeMillis(); 106 | for (ImmutableMap.Entry shard : shardMap.entrySet()) { 107 | jobMonitorImpl(runStartMillis, shard, maxAutoRetries); 108 | } 109 | } 110 | 111 | protected abstract void jobMonitorImpl(long runStartMillis, 112 | Map.Entry shard, 113 | int numAutoRetries); 114 | } 115 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/commons/healthcheck/ServerTracker.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.commons.healthcheck; 18 | 19 | import com.google.common.annotations.VisibleForTesting; 20 | import com.twitter.ostrich.stats.Stats; 21 | import org.slf4j.Logger; 22 | import org.slf4j.LoggerFactory; 23 | 24 | /** 25 | * Class that implements the server health status tracking. 26 | * 27 | * It keeps the way of how to do heartbeat to the server and also the current status of the server. 28 | * The algorithm of detecting server live/dead is based upon configurable consecutive 29 | * failures/successes. 30 | * 31 | * Only reportHeartbeatResult() API is synchronized so as to protect the consistency of internal 32 | * state. isLive() is not synchronized, and the visibility of ``isLive`` is guaranteed by using 33 | * volatile. 34 | */ 35 | public class ServerTracker { 36 | 37 | private static final Logger LOG = LoggerFactory.getLogger(ServerTracker.class); 38 | 39 | private final String host; 40 | private final int port; 41 | private final HeartBeater heartBeater; 42 | private final int consecutiveFailures; 43 | private final int consecutiveSuccesses; 44 | public final int pingIntervalSecs; 45 | private volatile boolean isLive = true; 46 | 47 | private int currentConsecutiveFailures; 48 | private int currentConsecutiveSuccesses; 49 | 50 | // Track how many simulated seconds have lapsed in test after this tracker is initialized. 51 | @VisibleForTesting 52 | public int totalSecondsLapsed = 0; 53 | 54 | public ServerTracker(String host, 55 | int port, 56 | HeartBeater heartBeater, 57 | int consecutiveFailures, 58 | int consecutiveSuccesses, 59 | int pingIntervalSecs, 60 | boolean isLiveInitially) { 61 | this.host = host; 62 | this.port = port; 63 | this.heartBeater = heartBeater; 64 | this.consecutiveFailures = consecutiveFailures; 65 | this.consecutiveSuccesses = consecutiveSuccesses; 66 | this.pingIntervalSecs = pingIntervalSecs; 67 | this.isLive = isLiveInitially; 68 | } 69 | 70 | /** 71 | * Ping the server. 72 | * 73 | * Returns: 74 | * True if the server is live. Otherwise either return False or throw an exception to 75 | * indicate a failed heartbeat 76 | */ 77 | public boolean heartbeat() throws Exception { 78 | return heartBeater.ping(); 79 | } 80 | 81 | /** 82 | * Report the result of a heartbeat. 83 | * 84 | * Args: 85 | * isLive: whether the heartbeat is successful. 86 | */ 87 | public void reportHeartbeatResult(boolean isLive) { 88 | if (isLive) { 89 | reportHeartbeatSuccess(); 90 | } else { 91 | reportHeartbeatFailure(); 92 | } 93 | } 94 | 95 | /** 96 | * Report a successful heartbeat. 97 | */ 98 | private synchronized void reportHeartbeatSuccess() { 99 | currentConsecutiveFailures = 0; 100 | currentConsecutiveSuccesses += 1; 101 | 102 | if (isLive) { 103 | return; 104 | } 105 | 106 | if (currentConsecutiveSuccesses >= consecutiveSuccesses) { 107 | LOG.info(String.format("Server %s:%d is determined as live by health check.", host, port)); 108 | isLive = true; 109 | } 110 | } 111 | 112 | /** 113 | * Report a failed heartbeat. 114 | */ 115 | private synchronized void reportHeartbeatFailure() { 116 | Stats.incr(String.format("heartbeat_failures_%s_%d", host, port)); 117 | currentConsecutiveSuccesses = 0; 118 | currentConsecutiveFailures += 1; 119 | 120 | if (!isLive) { 121 | // Do not LOG here to prevent noise. 122 | Stats.incr(String.format("healthcheck_dead_%s_%d", host, port)); 123 | return; 124 | } 125 | 126 | if (currentConsecutiveFailures >= consecutiveFailures) { 127 | LOG.info(String.format("Server %s:%d is determined as dead by health check.", host, port)); 128 | Stats.incr(String.format("healthcheck_dead_%s_%d", host, port)); 129 | isLive = false; 130 | } 131 | } 132 | 133 | /** 134 | * Returns whether the server is live now. 135 | */ 136 | public boolean isLive() { 137 | return isLive; 138 | } 139 | } 140 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/commons/util/BytesUtil.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.commons.util; 18 | 19 | import com.google.common.base.Charsets; 20 | import org.jboss.netty.buffer.ChannelBuffer; 21 | 22 | import java.nio.ByteBuffer; 23 | import java.util.Formatter; 24 | 25 | public class BytesUtil { 26 | 27 | /** 28 | * Converts a hex string to bytes. 29 | * 30 | * @param s a hex string 31 | * @return byte array 32 | */ 33 | public static byte[] hexStringToByteArray(String s) { 34 | int len = s.length(); 35 | byte[] data = new byte[len / 2]; 36 | for (int i = 0; i < len; i += 2) { 37 | data[i / 2] = (byte) ((Character.digit(s.charAt(i), 16) << 4) 38 | + Character.digit(s.charAt(i + 1), 16)); 39 | } 40 | return data; 41 | } 42 | 43 | /** 44 | * Converts a byte array to hex string. 45 | * 46 | * @param bytes a byte array. 47 | * @param pos the start position 48 | * @param len number of bytes to be converted 49 | * @return hex string. 50 | */ 51 | public static String byteArrayToHexString(byte[] bytes, int pos, int len) { 52 | StringBuilder sb = new StringBuilder(bytes.length * 2); 53 | Formatter formatter = new Formatter(sb); 54 | for (int i = pos; i < pos + len; ++i) { 55 | formatter.format("%02x", bytes[i]); 56 | } 57 | return sb.toString(); 58 | } 59 | 60 | /** 61 | * Reads the remaining bytes in a ByteBuffer into a byte[]. 62 | * 63 | * @param byteBuffer byte buffer to read from. 64 | * @return byte[] containing the bytes read. 65 | */ 66 | public static byte[] readBytesFromByteBuffer(ByteBuffer byteBuffer) { 67 | byte[] buffer = new byte[byteBuffer.remaining()]; 68 | byteBuffer.get(buffer); 69 | return buffer; 70 | } 71 | 72 | /** 73 | * Reads the remaining bytes in a ByteBuffer into a byte[] without consuming. 74 | * 75 | * @param byteBuffer byte buffer to read from. 76 | * @return byte[] containing the bytes read. 77 | */ 78 | public static byte[] readBytesFromByteBufferWithoutConsume(ByteBuffer byteBuffer) { 79 | byte[] buffer = new byte[byteBuffer.remaining()]; 80 | byteBuffer.duplicate().get(buffer); 81 | return buffer; 82 | } 83 | 84 | public static ByteBuffer stringToByteBuffer(String str) { 85 | return ByteBuffer.wrap(str.getBytes(Charsets.UTF_8)); 86 | } 87 | 88 | public static String stringFromByteBuffer(ByteBuffer byteBuf) { 89 | return new String(readBytesFromByteBuffer(byteBuf), Charsets.UTF_8); 90 | } 91 | 92 | public static ByteBuffer longToByteBuffer(long num) { 93 | return ByteBuffer.allocate(Long.SIZE / 8).putLong(0, num); 94 | } 95 | 96 | public static long longFromByteBuffer(ByteBuffer byteBuffer) { 97 | return ByteBuffer.wrap(readBytesFromByteBuffer(byteBuffer)).getLong(); 98 | } 99 | 100 | public static ByteBuffer intToByteBuffer(int num) { 101 | return ByteBuffer.allocate(Integer.SIZE / 8).putInt(0, num); 102 | } 103 | 104 | public static int intFromByteBuffer(ByteBuffer byteBuffer) { 105 | return ByteBuffer.wrap(readBytesFromByteBuffer(byteBuffer)).getInt(); 106 | } 107 | 108 | public static ByteBuffer doubleToByteBuffer(double val) { 109 | return ByteBuffer.allocate(Double.SIZE / 8).putDouble(val); 110 | } 111 | 112 | public static double doubleFromByteBuffer(ByteBuffer byteBuffer) { 113 | return ByteBuffer.wrap(readBytesFromByteBuffer(byteBuffer)).getDouble(); 114 | } 115 | 116 | public static ByteBuffer byteToByteBuffer(byte num) { 117 | return ByteBuffer.allocate(1).put(num); 118 | } 119 | 120 | public static byte byteFromByteBuffer(ByteBuffer byteBuffer) { 121 | return ByteBuffer.wrap(readBytesFromByteBuffer(byteBuffer)).get(); 122 | } 123 | 124 | public static byte[] toBytes(long num) { 125 | return readBytesFromByteBuffer(longToByteBuffer(num)); 126 | } 127 | 128 | public static byte[] toBytes(String str) { 129 | return readBytesFromByteBuffer(stringToByteBuffer(str)); 130 | } 131 | 132 | /** 133 | * Converts UNREAD BYTES to byte array from ChannelBuffer 134 | * NOTE: this will consume all the readable bytes from channel buffer 135 | */ 136 | public static byte[] toBytesWithoutConsume(ChannelBuffer input) { 137 | // save old reader's index & reset it 138 | int oldIndex = input.readerIndex(); 139 | input.resetReaderIndex(); 140 | // read bytes out 141 | byte[] output = new byte[input.readableBytes()]; 142 | input.readBytes(output); 143 | // set reader's index back to 144 | input.readerIndex(oldIndex); 145 | return output; 146 | } 147 | } 148 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/mysql/MySQLDataSources.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.mysql; 18 | 19 | import com.pinterest.pinlater.commons.jdbc.JdbcUtils; 20 | 21 | import org.apache.commons.configuration.PropertiesConfiguration; 22 | import org.apache.commons.dbcp.BasicDataSource; 23 | import org.slf4j.Logger; 24 | import org.slf4j.LoggerFactory; 25 | 26 | import java.sql.Connection; 27 | import java.sql.SQLException; 28 | import java.util.concurrent.TimeUnit; 29 | import java.util.concurrent.atomic.AtomicBoolean; 30 | import javax.sql.DataSource; 31 | 32 | /** 33 | * Helper class used by the PinLaterMySQLBackend to encapsulate JDBC data sources. 34 | */ 35 | public class MySQLDataSources { 36 | 37 | private static final Logger LOG = LoggerFactory.getLogger(MySQLDataSources.class); 38 | 39 | private final DataSource generalDataSource; 40 | private final DataSource monitorDataSource; 41 | private final String host; 42 | private final int port; 43 | private final String user; 44 | private final String passwd; 45 | private final AtomicBoolean dequeueOnly; 46 | 47 | public MySQLDataSources(PropertiesConfiguration configuration, String host, int port, 48 | String user, String passwd, boolean dequeueOnly) { 49 | int numGeneralConnections = configuration.getInt("BACKEND_CONNECTIONS_PER_SHARD"); 50 | int generalSocketTimeoutMillis = (int) TimeUnit.SECONDS.toMillis( 51 | configuration.getInt("BACKEND_SOCKET_TIMEOUT_SECONDS")); 52 | int numMonitorConnections = configuration.getInt("MONITOR_CONNECTIONS_PER_SHARD", 3); 53 | int monitorSocketTimeoutMillis = (int) TimeUnit.SECONDS.toMillis( 54 | configuration.getInt("MONITOR_SOCKET_TIMEOUT_SECONDS", 10)); 55 | int maxWaitMillis = configuration.getInt("BACKEND_CONNECTION_MAX_WAIT_MILLIS"); 56 | this.generalDataSource = createDataSource( 57 | host, port, user, passwd, numGeneralConnections, 58 | maxWaitMillis, generalSocketTimeoutMillis); 59 | this.monitorDataSource = createDataSource( 60 | host, port, user, passwd, numMonitorConnections, 61 | maxWaitMillis, monitorSocketTimeoutMillis); 62 | this.host = host; 63 | this.port = port; 64 | this.user = user; 65 | this.passwd = passwd; 66 | this.dequeueOnly = new AtomicBoolean(dequeueOnly); 67 | } 68 | 69 | public DataSource getGeneralDataSource() { return generalDataSource; } 70 | 71 | public DataSource getMonitorDataSource() { return monitorDataSource; } 72 | 73 | public boolean isDequeueOnly() { return dequeueOnly.get(); } 74 | 75 | public void setDequeueOnly(Boolean dequeueOnly) { this.dequeueOnly.set(dequeueOnly); } 76 | 77 | // NOTE: We don't check dequeueOnly here because it should be configurable while maintaining 78 | // the connection. 79 | public boolean needNewConnection(String host, int port, String user, String passwd) { 80 | return this.host.equals(host) && this.port == port 81 | && this.user.equals(user) && this.passwd.equals(passwd); 82 | } 83 | 84 | // NOTE: We intentionally clone this method from DataSourceUtil, since we want to carefully 85 | // control data source and ConnectorJ configuration. 86 | private static DataSource createDataSource( 87 | String host, int port, String user, String passwd, int poolSize, 88 | int maxWaitMillis, int socketTimeoutMillis) { 89 | BasicDataSource dataSource = new BasicDataSource(); 90 | dataSource.setDriverClassName("com.mysql.jdbc.Driver"); 91 | dataSource.setUrl(String.format( 92 | "jdbc:mysql://%s:%d?" 93 | + "connectTimeout=5000&" 94 | + "socketTimeout=%d&" 95 | + "enableQueryTimeouts=false&" 96 | + "cachePrepStmts=true&" 97 | + "characterEncoding=UTF-8", 98 | host, 99 | port, 100 | socketTimeoutMillis)); 101 | dataSource.setUsername(user); 102 | dataSource.setPassword(passwd); 103 | dataSource.setDefaultAutoCommit(true); 104 | dataSource.setInitialSize(poolSize); 105 | dataSource.setMaxActive(poolSize); 106 | dataSource.setMaxIdle(poolSize); 107 | // deal with idle connection eviction 108 | dataSource.setValidationQuery("SELECT 1 FROM DUAL"); 109 | dataSource.setTestOnBorrow(false); 110 | dataSource.setTestOnReturn(false); 111 | dataSource.setTestWhileIdle(true); 112 | dataSource.setMinEvictableIdleTimeMillis(5 * 60 * 1000); 113 | dataSource.setTimeBetweenEvictionRunsMillis(3 * 60 * 1000); 114 | dataSource.setNumTestsPerEvictionRun(poolSize); 115 | // max wait in milliseconds for a connection. 116 | dataSource.setMaxWait(maxWaitMillis); 117 | // force connection pool initialization. 118 | Connection conn = null; 119 | try { 120 | // Here not getting the connection from ThreadLocal no need to worry about that. 121 | conn = dataSource.getConnection(); 122 | } catch (SQLException e) { 123 | LOG.error( 124 | String.format("Failed to get a mysql connection when creating DataSource, " 125 | + "host: %s, port: %d", host, port), 126 | e); 127 | } finally { 128 | JdbcUtils.closeConnection(conn); 129 | } 130 | return dataSource; 131 | } 132 | } 133 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/common/PinLaterBackendUtils.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.common; 18 | 19 | import com.pinterest.pinlater.thrift.PinLaterJobInfo; 20 | 21 | import com.google.common.base.Preconditions; 22 | import com.google.common.collect.Iterables; 23 | import com.google.common.collect.Lists; 24 | import com.twitter.common.base.MorePreconditions; 25 | import com.twitter.util.Function; 26 | import com.twitter.util.Future; 27 | import org.apache.commons.codec.binary.Hex; 28 | 29 | import java.security.MessageDigest; 30 | import java.security.NoSuchAlgorithmException; 31 | import java.util.Comparator; 32 | import java.util.Iterator; 33 | import java.util.List; 34 | 35 | /** 36 | * Encapsulates static utility methods used by the different PinLater backend classes. 37 | */ 38 | public final class PinLaterBackendUtils { 39 | 40 | private static final String SALT = "VHzMjY2Kb9BYlyrbtpyFvnHVPZAvB5Rm"; 41 | 42 | /** 43 | * Get the salted hash of a password. 44 | * 45 | * @param password String to be salted and hashed. 46 | * @return Salted hash of the password string. 47 | */ 48 | public static String getSaltedHash(String password) throws NoSuchAlgorithmException { 49 | MessageDigest digest = MessageDigest.getInstance("SHA-256"); 50 | String strToDigest = password + SALT; 51 | byte[] digestedStr = digest.digest(strToDigest.getBytes()); 52 | return new String(Hex.encodeHex(digestedStr)); 53 | } 54 | 55 | /** 56 | * Executes a batch of requests asynchronously in a partitioned manner, 57 | * with the specified parallelism. 58 | * 59 | * @param requests List of requests to execute. 60 | * @param parallelism Desired parallelism (must be > 0). 61 | * @param executeBatch Function to execute each partitioned batch of requests. 62 | * @param Request type. 63 | * @param Response type. 64 | * @return List of response futures. 65 | */ 66 | public static List> executePartitioned( 67 | List requests, 68 | int parallelism, 69 | Function, Future> executeBatch) { 70 | MorePreconditions.checkNotBlank(requests); 71 | Preconditions.checkArgument(parallelism > 0); 72 | Preconditions.checkNotNull(executeBatch); 73 | 74 | int sizePerPartition = Math.max(requests.size() / parallelism, 1); 75 | List> partitions = Lists.partition(requests, sizePerPartition); 76 | List> futures = Lists.newArrayListWithCapacity(partitions.size()); 77 | for (final List request : partitions) { 78 | futures.add(executeBatch.apply(request)); 79 | } 80 | return futures; 81 | } 82 | 83 | /** 84 | * Merges multiple iterables into a list using the comparing logic provided by the comparator. 85 | * 86 | * @param iterablesToMerge The iterables to be merged. 87 | * @param comparator Comparator specifying the comparison logic between the iterables. 88 | * @param Iterable item type. 89 | * @param Comparator between iterate items type. 90 | * @return List of all merged results. 91 | */ 92 | public static > List mergeIntoList( 93 | Iterable> iterablesToMerge, 94 | S comparator) { 95 | return mergeIntoList(iterablesToMerge, comparator, Integer.MAX_VALUE); 96 | } 97 | 98 | /** 99 | * Merges multiple iterables into a list using the comparing logic provided by the comparator. 100 | * The returned list will only include the first n merged items, where n is the limit specified. 101 | * 102 | * @param iterablesToMerge The iterables to be merged. 103 | * @param comparator Comparator specifying the comparison logic between the iterables. 104 | * @param limit Max number of results that will be returned. 105 | * @param Iterable item type. 106 | * @param Comparator between iterate items type. 107 | * @return List of the first n merged results. 108 | */ 109 | public static > List mergeIntoList( 110 | Iterable> iterablesToMerge, 111 | S comparator, 112 | int limit) { 113 | // Perform a k-way merge on the collections and return the result in an ArrayList. 114 | List mergedCols = Lists.newLinkedList(); 115 | Iterator mergeIterator = Iterables.mergeSorted(iterablesToMerge, comparator).iterator(); 116 | while (mergeIterator.hasNext() && mergedCols.size() < limit) { 117 | mergedCols.add(mergeIterator.next()); 118 | } 119 | return mergedCols; 120 | } 121 | 122 | /** 123 | * Comparator for PinLaterJobInfo: descending in runAfter. 124 | */ 125 | public static class JobInfoComparator implements Comparator { 126 | 127 | private static JobInfoComparator INSTANCE = new JobInfoComparator(); 128 | 129 | private JobInfoComparator() {} 130 | 131 | public static JobInfoComparator getInstance() { 132 | return INSTANCE; 133 | } 134 | 135 | public int compare(PinLaterJobInfo a, PinLaterJobInfo b) { 136 | Long longA = new Long(a.getRunAfterTimestampMillis()); 137 | Long longB = new Long(b.getRunAfterTimestampMillis()); 138 | return -1 * longA.compareTo(longB); 139 | } 140 | } 141 | } 142 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/commons/ostrich/StatTrackingEventListener.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.commons.ostrich; 18 | 19 | import com.google.common.annotations.VisibleForTesting; 20 | import com.google.common.base.Predicate; 21 | import com.google.common.base.Predicates; 22 | import com.google.common.base.Strings; 23 | import com.google.common.collect.Maps; 24 | import com.pinterest.pinlater.commons.util.TimeUtils; 25 | import com.twitter.common.base.MorePreconditions; 26 | import com.twitter.ostrich.stats.Stats; 27 | import com.twitter.util.FutureEventListener; 28 | import org.slf4j.Logger; 29 | import org.slf4j.LoggerFactory; 30 | 31 | import java.util.Map; 32 | 33 | /** 34 | * An event listener for tracking success times and failure exceptions for futures. 35 | * @param The inner class of the future. 36 | */ 37 | public class StatTrackingEventListener implements FutureEventListener { 38 | 39 | private static final Logger LOG = LoggerFactory.getLogger(StatTrackingEventListener.class); 40 | 41 | private long startTime; 42 | private String statPrefix; 43 | private String info; 44 | private boolean logError; 45 | private Predicate ignoreExceptionPredicate; 46 | private Map tags; 47 | 48 | /** 49 | * An event listener for tracking success and failure times and exceptions. 50 | * @param startTime Obtained from TimeUtils.millisTime() ONLY! 51 | * @param prefix The prefix of the stat. 52 | * @param info The info will be logged when an exception happens 53 | * @param logError true if errors should be logged 54 | * @param tags a map of tags to be appended onto the stats 55 | */ 56 | public StatTrackingEventListener( 57 | long startTime, 58 | String prefix, 59 | String info, 60 | boolean logError, 61 | Map tags) { 62 | this.startTime = startTime; 63 | this.statPrefix = MorePreconditions.checkNotBlank(prefix); 64 | this.info = info; 65 | this.logError = logError; 66 | ignoreExceptionPredicate = Predicates.alwaysFalse(); 67 | appendTags(tags); 68 | } 69 | 70 | /** 71 | * An event listener for tracking success and failure times and exceptions. 72 | * @param startTime Obtained from TimeUtils.millisTime() ONLY! 73 | * @param prefix The prefix of the stat. 74 | * @param info The info will be logged when an exception happens 75 | * @param logError true if errors should be logged 76 | */ 77 | public StatTrackingEventListener( 78 | long startTime, 79 | String prefix, 80 | String info, 81 | boolean logError) { 82 | this(startTime, prefix, info, logError, null); 83 | } 84 | 85 | /** 86 | * An event listener for tracking success and failure times and exceptions. 87 | * @param startTime Obtained from TimeUtils.millisTime() ONLY! 88 | * @param prefix The prefix of the stat. 89 | * @param logError true if errors should be logged 90 | */ 91 | public StatTrackingEventListener(long startTime, String prefix, boolean logError) { 92 | this(startTime, prefix, prefix, logError, null); 93 | } 94 | 95 | /** 96 | * An event listener for tracking success and failure times and exceptions. 97 | * The error message won't be logged 98 | * @param startTime Obtained from TimeUtils.millisTime() ONLY! 99 | * @param prefix The prefix of the stat. 100 | */ 101 | public StatTrackingEventListener(long startTime, String prefix) { 102 | this(startTime, prefix, null, false, null); 103 | } 104 | 105 | @Override 106 | public void onSuccess(T t) { 107 | // Counts can be derived from the metric's count. 108 | Stats.addMetric( 109 | String.format("%s.success", statPrefix), 110 | (int) (TimeUtils.millisTime() - startTime)); 111 | String tagsString = getTagsString(); 112 | if (!Strings.isNullOrEmpty(tagsString)) { 113 | Stats.addMetric( 114 | String.format("%s.success.withtags%s", statPrefix, tagsString), 115 | (int) (TimeUtils.millisTime() - startTime)); 116 | } 117 | } 118 | 119 | @Override 120 | public void onFailure(Throwable throwable) { 121 | boolean isExceptionIgnored = ignoreExceptionPredicate.apply(throwable); 122 | if (logError && !isExceptionIgnored) { 123 | int duration = (int) (TimeUtils.millisTime() - startTime); 124 | String message = String.format("%s; Duration: %s; %s", statPrefix, duration, info); 125 | LOG.error(message, throwable); 126 | } 127 | 128 | String exceptionStatsString = isExceptionIgnored ? "ignoredexception" : "exception"; 129 | Stats.incr(String.format("%s.%s%s", statPrefix, exceptionStatsString, getTagsString())); 130 | Stats.incr( 131 | String.format( 132 | "%s.%s.%s%s", 133 | statPrefix, 134 | exceptionStatsString, 135 | throwable.getClass().getSimpleName(), 136 | getTagsString())); 137 | } 138 | 139 | /** 140 | * Append a map of tag names and tag values to the end of the tags string. 141 | * The tags string will be attached onto counters/metrics 142 | * @param newTags a map of tag names and tag values 143 | */ 144 | public StatTrackingEventListener appendTags(Map newTags) { 145 | if (newTags == null || newTags.isEmpty()) { 146 | return this; 147 | } 148 | if (tags == null) { 149 | tags = Maps.newLinkedHashMap(); 150 | } 151 | tags.putAll(newTags); 152 | return this; 153 | } 154 | 155 | /** 156 | * Get a tag string from tags map to append to the stat name 157 | */ 158 | @VisibleForTesting 159 | public String getTagsString() { 160 | if (tags == null || tags.isEmpty()) { 161 | return ""; 162 | } 163 | StringBuilder tagBuilder = new StringBuilder(); 164 | for (Map.Entry tag : tags.entrySet()) { 165 | if (tag.getValue() != null && !tag.getValue().isEmpty()) { 166 | tagBuilder 167 | .append(" ") 168 | .append(tag.getKey()) 169 | .append("=") 170 | .append(tag.getValue()); 171 | } 172 | } 173 | return tagBuilder.toString(); 174 | } 175 | 176 | public StatTrackingEventListener setIgnoreExceptionPredicate(Predicate predicate) { 177 | if (predicate != null) { 178 | ignoreExceptionPredicate = predicate; 179 | } else { 180 | LOG.error("Null ignoreExceptionPredicate! Replaced with Predicates.alwaysFalse()"); 181 | ignoreExceptionPredicate = Predicates.alwaysFalse(); 182 | } 183 | return this; 184 | } 185 | } 186 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/mysql/MySQLQueueMonitor.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.mysql; 18 | 19 | import com.pinterest.pinlater.backends.common.BackendQueueMonitorBase; 20 | import com.pinterest.pinlater.commons.jdbc.JdbcUtils; 21 | import com.pinterest.pinlater.thrift.PinLaterJobState; 22 | 23 | import com.google.common.annotations.VisibleForTesting; 24 | import com.google.common.collect.ImmutableMap; 25 | import com.twitter.ostrich.stats.Stats; 26 | import org.apache.commons.configuration.PropertiesConfiguration; 27 | import org.slf4j.Logger; 28 | import org.slf4j.LoggerFactory; 29 | 30 | import java.sql.Connection; 31 | import java.sql.ResultSet; 32 | import java.sql.Timestamp; 33 | import java.util.Map; 34 | import java.util.Set; 35 | 36 | /** 37 | * Implements a scheduled task used by the PinLaterMySQLBackend for various types of 38 | * job queue cleanup, including ACK timeouts and GC'ing finished jobs. 39 | */ 40 | public class MySQLQueueMonitor extends BackendQueueMonitorBase { 41 | 42 | private static final Logger LOG = LoggerFactory.getLogger(PinLaterMySQLBackend.class); 43 | private int logCount; 44 | private int numTimeoutDone; 45 | private int numTimeoutRetry; 46 | private int numSucceededGC; 47 | private int numFailedGC; 48 | 49 | public MySQLQueueMonitor(ImmutableMap shardMap, 50 | PropertiesConfiguration configuration) { 51 | super(shardMap, configuration); 52 | } 53 | 54 | @VisibleForTesting 55 | public MySQLQueueMonitor(ImmutableMap shardMap, 56 | int updateMaxSize, int maxAutoRetries, int logInterval, 57 | long jobClaimedTimeoutMillis, long jobSucceededGCTimeoutMillis, 58 | long jobFailedGCTimeoutMillis, 59 | int numPriorityLevels) { 60 | super(shardMap, updateMaxSize, maxAutoRetries, logInterval, jobClaimedTimeoutMillis, 61 | jobSucceededGCTimeoutMillis, jobFailedGCTimeoutMillis, numPriorityLevels); 62 | } 63 | 64 | @Override 65 | public void run() { 66 | final long runStartMillis = System.currentTimeMillis(); 67 | for (ImmutableMap.Entry shard : getShardMap().entrySet()) { 68 | jobMonitorImpl(runStartMillis, shard, getMaxAutoRetries()); 69 | } 70 | } 71 | 72 | @Override 73 | protected void jobMonitorImpl(long runStartMillis, 74 | Map.Entry shard, 75 | int numAutoRetries) { 76 | Connection conn = null; 77 | ResultSet dbNameRs = null; 78 | Timestamp succeededGCTimestamp = new Timestamp( 79 | runStartMillis - getJobSucceededGCTimeoutMillis()); 80 | Timestamp failedGCTimestamp = new Timestamp(runStartMillis - getJobFailedGCTimeoutMillis()); 81 | Timestamp timeoutTimestamp = new Timestamp(runStartMillis - getJobClaimedTimeoutMillis()); 82 | 83 | try { 84 | conn = shard.getValue().getMonitorDataSource().getConnection(); 85 | Set queueNames = MySQLBackendUtils.getQueueNames(conn, shard.getKey()); 86 | for (String queueName : queueNames) { 87 | for (int priority = 1; 88 | priority <= numPriorityLevels; 89 | priority++) { 90 | String jobsTableName = MySQLBackendUtils.constructJobsTableName( 91 | queueName, shard.getKey(), priority); 92 | 93 | // Handle timed out jobs with attempts exhausted. 94 | numTimeoutDone += JdbcUtils.executeUpdate( 95 | conn, 96 | String.format(MySQLQueries.MONITOR_TIMEOUT_DONE_UPDATE, jobsTableName), 97 | PinLaterJobState.FAILED.getValue(), 98 | PinLaterJobState.IN_PROGRESS.getValue(), 99 | timeoutTimestamp, 100 | getUpdateMaxSize()); 101 | 102 | // Handle timed out jobs with attempts remaining. 103 | numTimeoutRetry += JdbcUtils.executeUpdate( 104 | conn, 105 | String.format(MySQLQueries.MONITOR_TIMEOUT_RETRY_UPDATE, jobsTableName), 106 | PinLaterJobState.PENDING.getValue(), 107 | PinLaterJobState.IN_PROGRESS.getValue(), 108 | timeoutTimestamp, 109 | getUpdateMaxSize()); 110 | 111 | // Succeeded job GC. 112 | numSucceededGC += JdbcUtils.executeUpdate( 113 | conn, 114 | String.format(MySQLQueries.MONITOR_GC_DONE_JOBS, jobsTableName), 115 | PinLaterJobState.SUCCEEDED.getValue(), 116 | succeededGCTimestamp, 117 | getUpdateMaxSize()); 118 | 119 | // Failed job GC. 120 | numFailedGC += JdbcUtils.executeUpdate( 121 | conn, 122 | String.format(MySQLQueries.MONITOR_GC_DONE_JOBS, jobsTableName), 123 | PinLaterJobState.FAILED.getValue(), 124 | failedGCTimestamp, 125 | getUpdateMaxSize()); 126 | 127 | logCount++; 128 | if (logCount % getLogInterval() == 0) { 129 | LOG.info(String.format( 130 | "JobQueueMonitor: " 131 | + "Shard: %s Queue: %s Priority: %d Timeout Done: %d Timeout Retry: %d " 132 | + "Succeeded GC: %d Failed GC: %d", 133 | shard.getKey(), queueName, priority, numTimeoutDone, numTimeoutRetry, 134 | numSucceededGC, numFailedGC)); 135 | Stats.incr(queueName + "_timeout_done", numTimeoutDone); 136 | Stats.incr(queueName + "_timeout_retry", numTimeoutRetry); 137 | Stats.incr(queueName + "_succeeded_gc", numSucceededGC); 138 | Stats.incr(queueName + "_failed_gc", numFailedGC); 139 | logCount = 0; 140 | numTimeoutDone = 0; 141 | numTimeoutRetry = 0; 142 | numSucceededGC = 0; 143 | numFailedGC = 0; 144 | } 145 | } 146 | } 147 | } catch (Exception e) { 148 | // Deadlocks are occasionally expected for our high-contention queries. We 149 | // retry a few times so as not to abort an entire monitor cycle. 150 | if (MySQLBackendUtils.isDeadlockException(e) && numAutoRetries > 0) { 151 | Stats.incr("mysql-deadlock-monitor"); 152 | jobMonitorImpl(runStartMillis, shard, numAutoRetries - 1); 153 | } else { 154 | LOG.error("Exception in JobQueueMonitor task", e); 155 | } 156 | } finally { 157 | JdbcUtils.closeResultSet(dbNameRs); 158 | JdbcUtils.closeConnection(conn); 159 | } 160 | } 161 | } 162 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/mysql/MySQLHealthMonitor.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.mysql; 18 | 19 | import com.google.common.base.Preconditions; 20 | import com.google.common.collect.ImmutableMap; 21 | import org.slf4j.Logger; 22 | import org.slf4j.LoggerFactory; 23 | 24 | import java.util.Map; 25 | import java.util.Set; 26 | import java.util.concurrent.LinkedBlockingDeque; 27 | import java.util.concurrent.TimeUnit; 28 | import java.util.concurrent.atomic.AtomicReference; 29 | 30 | /** 31 | * Class that monitors the health of MySQL shards. The basic strategy is to keep a recent 32 | * window of "health" samples, i.e. whether a query was a success or failure, and use a 33 | * threshold to determine if the shard should be marked unhealthy. Once marked unhealthy, 34 | * status is reset after a short period of time, and the sampling process is repeated 35 | * from scratch. 36 | * 37 | * This class is thread safe. 38 | */ 39 | public class MySQLHealthMonitor { 40 | 41 | private static final Logger LOG = LoggerFactory.getLogger(MySQLHealthMonitor.class); 42 | 43 | private final int windowSize; 44 | private final int healthySuccessThreshold; 45 | private final long unhealthyResetPeriodMillis; 46 | // Note: ShardState access should be explicitly synchronized. 47 | private final AtomicReference> shardHealthMapRef = 48 | new AtomicReference>(); 49 | 50 | /** 51 | * Creates a MySQLHealthMonitor object with default configuration. 52 | * 53 | * @param shardNames set of MySQL shard ids. 54 | */ 55 | public MySQLHealthMonitor(Set shardNames) { 56 | this(shardNames, 57 | 100, // window size 58 | 75, // healthy success threshold 59 | TimeUnit.SECONDS.toMillis(60)); // health reset period 60 | } 61 | 62 | /** 63 | * Creates a MySQLHealthMonitor object. 64 | * 65 | * @param shardNames set of MySQL shard names. 66 | * @param windowSize number of recent health samples to keep per shard. 67 | * @param healthySuccessThreshold number of samples in windowSize that should successful for 68 | * marking a shard healthy. 69 | * @param unhealthyResetPeriodMillis period of time after which to reset unhealthy status. 70 | */ 71 | public MySQLHealthMonitor(Set shardNames, 72 | int windowSize, 73 | int healthySuccessThreshold, 74 | long unhealthyResetPeriodMillis) { 75 | this.windowSize = windowSize; 76 | Preconditions.checkArgument(windowSize > 0); 77 | this.healthySuccessThreshold = healthySuccessThreshold; 78 | Preconditions.checkArgument( 79 | healthySuccessThreshold >= 0 && healthySuccessThreshold <= windowSize); 80 | this.unhealthyResetPeriodMillis = unhealthyResetPeriodMillis; 81 | Preconditions.checkArgument(unhealthyResetPeriodMillis > 0); 82 | 83 | ImmutableMap.Builder builder = 84 | new ImmutableMap.Builder(); 85 | for (String shard : shardNames) { 86 | builder.put(shard, new ShardState(windowSize)); 87 | } 88 | this.shardHealthMapRef.set(builder.build()); 89 | } 90 | 91 | public synchronized void recordSample(String shardName, boolean success) { 92 | ImmutableMap shardHealthMap = shardHealthMapRef.get(); 93 | 94 | ShardState shardState = shardHealthMap.get(shardName); 95 | if (shardState.healthSamples.remainingCapacity() == 0) { 96 | shardState.numSuccessesInWindow -= shardState.healthSamples.removeLast(); 97 | } 98 | 99 | int successVal = success ? 1 : 0; 100 | shardState.numSuccessesInWindow += successVal; 101 | shardState.healthSamples.addFirst(successVal); 102 | } 103 | 104 | public synchronized boolean isHealthy(String shardName) { 105 | long currentTimeMillis = System.currentTimeMillis(); 106 | ImmutableMap shardHealthMap = shardHealthMapRef.get(); 107 | if (!shardHealthMap.containsKey(shardName)) { 108 | LOG.error("Health monitor map doesn't have shard name {}", shardName); 109 | return false; 110 | } 111 | 112 | ShardState shardState = shardHealthMap.get(shardName); 113 | if (shardState.isHealthy) { 114 | if (shardState.healthSamples.remainingCapacity() == 0 115 | && shardState.numSuccessesInWindow < healthySuccessThreshold) { 116 | LOG.info("Marking mysql shard {} unhealthy due to {} successes / 100", 117 | shardName, shardState.numSuccessesInWindow); 118 | shardState.isHealthy = false; 119 | shardState.lastUnhealthyTimestampMillis = currentTimeMillis; 120 | } 121 | } else if (currentTimeMillis 122 | >= shardState.lastUnhealthyTimestampMillis + unhealthyResetPeriodMillis) { 123 | LOG.info("Resetting health state for mysql shard {}", shardName); 124 | shardState.reset(); 125 | } 126 | 127 | return shardState.isHealthy; 128 | } 129 | 130 | /** 131 | * Update health monitor map when the shard config is updated. This method doesn't need to be 132 | * synchronized because it is only called when config is updated, and ConfigFileWatcher ensures 133 | * that only one update is active. 134 | * @param shardNamesToChanged map of shard names, the value is true when it's a new/changed shard 135 | */ 136 | public void updateHealthMap(Map shardNamesToChanged) { 137 | ImmutableMap.Builder builder = 138 | new ImmutableMap.Builder(); 139 | for (String shard : shardNamesToChanged.keySet()) { 140 | if (!shardNamesToChanged.get(shard)) { 141 | builder.put(shard, shardHealthMapRef.get().get(shard)); 142 | } else { 143 | builder.put(shard, new ShardState(windowSize)); 144 | } 145 | } 146 | 147 | shardHealthMapRef.set(builder.build()); 148 | } 149 | 150 | /** 151 | * Encapsulates per-MySQL-shard state kept track of by this class. 152 | */ 153 | private static class ShardState { 154 | 155 | final LinkedBlockingDeque healthSamples; 156 | int numSuccessesInWindow = 0; 157 | boolean isHealthy = true; 158 | long lastUnhealthyTimestampMillis = 0L; 159 | 160 | ShardState(int windowSize) { 161 | this.healthSamples = new LinkedBlockingDeque(windowSize); 162 | } 163 | 164 | public void reset() { 165 | numSuccessesInWindow = 0; 166 | isHealthy = true; 167 | lastUnhealthyTimestampMillis = 0; 168 | healthSamples.clear(); 169 | } 170 | } 171 | } 172 | -------------------------------------------------------------------------------- /src/test/java/com/pinterest/pinlater/backends/redis/RedisQueueMonitorTest.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.redis; 18 | 19 | import com.pinterest.pinlater.commons.healthcheck.DummyHeartBeater; 20 | import com.pinterest.pinlater.commons.healthcheck.HealthChecker; 21 | import com.pinterest.pinlater.backends.common.BackendQueueMonitorBaseTest; 22 | import com.pinterest.pinlater.backends.common.PinLaterTestUtils; 23 | import com.pinterest.pinlater.thrift.PinLaterDequeueRequest; 24 | import com.pinterest.pinlater.thrift.PinLaterDequeueResponse; 25 | import com.pinterest.pinlater.thrift.PinLaterEnqueueRequest; 26 | import com.pinterest.pinlater.thrift.PinLaterEnqueueResponse; 27 | import com.pinterest.pinlater.thrift.PinLaterJob; 28 | import com.pinterest.pinlater.thrift.PinLaterJobState; 29 | 30 | import com.google.common.collect.ImmutableMap; 31 | import org.apache.commons.configuration.ConfigurationException; 32 | import org.apache.commons.configuration.PropertiesConfiguration; 33 | import org.junit.After; 34 | import org.junit.Assert; 35 | import org.junit.Assume; 36 | import org.junit.Before; 37 | import org.junit.BeforeClass; 38 | import org.junit.Test; 39 | import org.junit.runner.RunWith; 40 | import org.junit.runners.JUnit4; 41 | 42 | import java.io.InputStream; 43 | import java.nio.ByteBuffer; 44 | import java.util.concurrent.TimeUnit; 45 | 46 | @RunWith(JUnit4.class) 47 | public class RedisQueueMonitorTest extends BackendQueueMonitorBaseTest { 49 | 50 | private static final String QUEUE_NAME = "redis_queue_monitor_test"; 51 | private static final String LOCALHOST = "localhost"; 52 | private static final int REDIS_PORT = 6379; 53 | 54 | private static PinLaterRedisBackend backend; 55 | private static PropertiesConfiguration configuration; 56 | 57 | @Override 58 | protected String getQueueName() { 59 | return QUEUE_NAME; 60 | } 61 | 62 | @Override 63 | protected PinLaterRedisBackend getBackend() { 64 | return backend; 65 | } 66 | 67 | @Override 68 | protected RedisQueueMonitor createQueueMonitor(long jobClaimedTimeoutMillis, 69 | long jobSucceededGCTimeoutMillis, 70 | long jobFailedGCTimeoutMillis) { 71 | HealthChecker dummyHealthChecker = new HealthChecker("pinlater_redis_test"); 72 | dummyHealthChecker.addServer( 73 | LOCALHOST, 74 | REDIS_PORT, 75 | new DummyHeartBeater(true), 76 | 10, // consecutive failures 77 | 10, // consecutive successes 78 | 10, // ping interval secs 79 | true); // is live initially 80 | 81 | InputStream redisConfigStream = ClassLoader.getSystemResourceAsStream("redis.local.json"); 82 | ImmutableMap shardMap = 83 | RedisBackendUtils.buildShardMap(redisConfigStream, configuration); 84 | return new RedisQueueMonitor(shardMap, 85 | 1000, // update max size 86 | 3, // auto retries 87 | 1, // log interval 88 | jobClaimedTimeoutMillis, 89 | jobSucceededGCTimeoutMillis, 90 | jobFailedGCTimeoutMillis, 91 | 3, // max queue priority 92 | dummyHealthChecker 93 | ); 94 | } 95 | 96 | @BeforeClass 97 | public static void beforeClass() throws Exception { 98 | // If there is no local Redis, skip this test. 99 | Assume.assumeTrue(LocalRedisChecker.isRunning(REDIS_PORT)); 100 | 101 | // If there is no local Redis, skip this test. 102 | Assume.assumeTrue(LocalRedisChecker.isRunning(REDIS_PORT)); 103 | 104 | configuration = new PropertiesConfiguration(); 105 | try { 106 | configuration.load(ClassLoader.getSystemResourceAsStream("pinlater.redis.test.properties")); 107 | } catch (ConfigurationException e) { 108 | throw new RuntimeException(e); 109 | } 110 | InputStream redisConfigStream = ClassLoader.getSystemResourceAsStream("redis.local.json"); 111 | 112 | backend = new PinLaterRedisBackend( 113 | configuration, redisConfigStream, "localhost", System.currentTimeMillis()); 114 | } 115 | 116 | @Before 117 | public void beforeTest() { 118 | // Clean up redis. 119 | backend.cleanUpAllShards().get(); 120 | 121 | // Create the test queue. 122 | backend.createQueue(QUEUE_NAME).get(); 123 | } 124 | 125 | @After 126 | public void afterTest() { 127 | if (!LocalRedisChecker.isRunning(REDIS_PORT)) { 128 | return; 129 | } 130 | 131 | // Clean up redis. 132 | backend.cleanUpAllShards().get(); 133 | } 134 | 135 | @Test 136 | public void testTimeoutJobHashEvicted() throws InterruptedException { 137 | // Ensure that queue monitor handles gracefully when the hash of the job is evicted but job id 138 | // is still in the in progress queue. 139 | Assert.assertTrue(backend.getQueueNames().get().contains(QUEUE_NAME)); 140 | 141 | // Enqueue 8 jobs, 4 of them with no retries, and 4 with 1 retry. 142 | PinLaterEnqueueRequest enqueueRequest = new PinLaterEnqueueRequest(); 143 | enqueueRequest.setQueueName(getQueueName()); 144 | for (int i = 0; i < 8; i++) { 145 | PinLaterJob job = new PinLaterJob(ByteBuffer.wrap(new String("job_body_" + i).getBytes())); 146 | job.setNumAttemptsAllowed(i < 4 ? 1 : 2); 147 | enqueueRequest.addToJobs(job); 148 | } 149 | PinLaterEnqueueResponse enqueueResponse = getBackend().enqueueJobs(enqueueRequest).get(); 150 | Assert.assertEquals(8, enqueueResponse.getJobDescriptorsSize()); 151 | 152 | // Dequeue all 8. 153 | PinLaterDequeueRequest dequeueRequest = new PinLaterDequeueRequest(getQueueName(), 8); 154 | PinLaterDequeueResponse 155 | dequeueResponse = 156 | getBackend().dequeueJobs("test", dequeueRequest).get(); 157 | Assert.assertEquals(8, PinLaterTestUtils.getJobCount(getBackend(), getQueueName(), 158 | PinLaterJobState.IN_PROGRESS)); 159 | 160 | Thread.sleep(TimeUnit.SECONDS.toMillis(1)); 161 | 162 | // Remove the job hash for the first 2 jobs to simulate they are evicted. Queue monitor should 163 | // just silently remove them from in process queue without adding them to any other queues. 164 | for (int i = 0; i < 2; i++) { 165 | getBackend().removeJobHash(enqueueResponse.getJobDescriptors().get(i)).get(); 166 | } 167 | 168 | // Run queue monitor configured with low job claimed timeout. 169 | createQueueMonitor(1, TimeUnit.HOURS.toMillis(1), TimeUnit.HOURS.toMillis(1)).run(); 170 | 171 | // After GC, 2 jobs should go to FAILED, 2 jobs(hash evicted) should disappear from any queue, 172 | // and remaining 4 should go to SUCCEEDED. 173 | Assert.assertEquals(2, PinLaterTestUtils.getJobCount(getBackend(), getQueueName(), 174 | PinLaterJobState.FAILED)); 175 | Assert.assertEquals(4, PinLaterTestUtils.getJobCount(getBackend(), getQueueName(), 176 | PinLaterJobState.PENDING)); 177 | } 178 | } 179 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/commons/config/ConfigFileServerSet.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.commons.config; 18 | 19 | import com.google.common.annotations.VisibleForTesting; 20 | import com.google.common.base.Preconditions; 21 | import com.google.common.collect.ImmutableSet; 22 | import com.twitter.common.base.Command; 23 | import com.twitter.common.base.MorePreconditions; 24 | import com.twitter.common.zookeeper.Group; 25 | import com.twitter.common.zookeeper.ServerSet; 26 | import com.twitter.thrift.Endpoint; 27 | import com.twitter.thrift.ServiceInstance; 28 | import com.twitter.thrift.Status; 29 | import com.twitter.util.ExceptionalFunction; 30 | import org.apache.commons.lang.StringUtils; 31 | import org.slf4j.Logger; 32 | import org.slf4j.LoggerFactory; 33 | 34 | import java.io.BufferedReader; 35 | import java.io.ByteArrayInputStream; 36 | import java.io.File; 37 | import java.io.IOException; 38 | import java.io.InputStream; 39 | import java.io.InputStreamReader; 40 | import java.net.InetSocketAddress; 41 | import java.util.Collections; 42 | import java.util.Map; 43 | 44 | /** 45 | * Implementation of the ServerSet interface that uses a file on local disk instead of talking 46 | * to ZooKeeper directly. The file is typically downloaded by an external daemon process that 47 | * registers a watch on the actual ZooKeeper server set. Consumers of this class simply need to 48 | * provide the ZooKeeper server set path. All other details are handled automatically, since the 49 | * external daemon is configured to use a standard and consistent file path scheme. 50 | * 51 | * Note that this implementation only supports monitor() and not join(). Use the standard 52 | * ZooKeeper implementation for join(). 53 | * 54 | */ 55 | public class ConfigFileServerSet implements ServerSet { 56 | 57 | private static Logger LOG = LoggerFactory.getLogger(ConfigFileServerSet.class); 58 | 59 | protected final ConfigFileWatcher configFileWatcher; 60 | protected final String serverSetFilePath; 61 | 62 | /** 63 | * Create a ConfigFileServerSet instance. 64 | * 65 | * @param serverSetZKPath ZooKeeper server set path 66 | */ 67 | public ConfigFileServerSet(String serverSetZKPath) { 68 | this(ConfigFileWatcher.defaultInstance(), serverSetZKPath); 69 | } 70 | 71 | /** 72 | * Internal constructor. This is provided for use by unit test. 73 | * 74 | * @param configFileWatcher ConfigFileWatcher instance to use. 75 | * @param serverSetFilePath Path the server set file on local disk. This is expected to contain 76 | * a list of host:port pairs, one per line. An external daemon will be 77 | * responsible for keeping this in sync with the actual server set in 78 | * ZooKeeper. 79 | */ 80 | @VisibleForTesting 81 | ConfigFileServerSet(ConfigFileWatcher configFileWatcher, String serverSetFilePath) { 82 | this.serverSetFilePath = MorePreconditions.checkNotBlank(serverSetFilePath); 83 | this.configFileWatcher = Preconditions.checkNotNull(configFileWatcher); 84 | 85 | File file = new File(serverSetFilePath); 86 | if (!file.exists()) { 87 | String message = String.format("Server set file: %s doesn't exist", serverSetFilePath); 88 | throw new IllegalArgumentException(message); 89 | } 90 | } 91 | 92 | @Override 93 | public EndpointStatus join( 94 | InetSocketAddress endpoint, Map additionalEndpoints, Status status) 95 | throws Group.JoinException, InterruptedException { 96 | throw new UnsupportedOperationException("ConfigFileServerSet does not support join()"); 97 | } 98 | 99 | @Override 100 | public EndpointStatus join( 101 | InetSocketAddress endpoint, Map additionalEndpoints) 102 | throws Group.JoinException, InterruptedException { 103 | throw new UnsupportedOperationException("ConfigFileServerSet does not support join()"); 104 | } 105 | 106 | @Override 107 | public EndpointStatus join( 108 | InetSocketAddress endpoint, Map additionalEndpoints, int shardId) 109 | throws Group.JoinException, InterruptedException { 110 | throw new UnsupportedOperationException("ConfigFileServerSet does not support join()"); 111 | } 112 | 113 | @Override 114 | public Command watch(final HostChangeMonitor monitor) throws MonitorException { 115 | monitor(monitor); 116 | return null; 117 | } 118 | 119 | @Override 120 | public void monitor(final HostChangeMonitor monitor) throws MonitorException { 121 | Preconditions.checkNotNull(monitor); 122 | try { 123 | // Each call to monitor registers a new file watch. This is a bit inefficient if there 124 | // are many calls to monitor(), since each watch needs to parse the file contents 125 | // independently. But it is simple and delegates keeping track of a list of monitors to the 126 | // ConfigFileWatcher. In practice, we don't really expect multiple calls to monitor anyway. 127 | configFileWatcher.addWatch(serverSetFilePath, 128 | new ExceptionalFunction() { 129 | @Override 130 | public Void applyE(byte[] newContents) throws Exception { 131 | ImmutableSet newServerSet = readServerSet(newContents); 132 | monitor.onChange(newServerSet); 133 | return null; 134 | } 135 | }); 136 | } catch (IOException e) { 137 | throw new MonitorException( 138 | "Error setting up watch on dynamic server set file:" + serverSetFilePath, e); 139 | } 140 | } 141 | 142 | protected static ImmutableSet readServerSet(byte[] fileContent) 143 | throws IOException { 144 | ImmutableSet.Builder builder = new ImmutableSet.Builder(); 145 | InputStream stream = new ByteArrayInputStream(fileContent); 146 | BufferedReader reader = new BufferedReader(new InputStreamReader(stream)); 147 | while (true) { 148 | String line = reader.readLine(); 149 | if (line == null) { 150 | // EOF. 151 | break; 152 | } else if (line.isEmpty()) { 153 | // Skip empty lines. 154 | continue; 155 | } 156 | 157 | // We expect each line to be of the form "hostname:port". Note that host names can 158 | // contain ':' themselves (e.g. ipv6 addresses). 159 | int index = line.lastIndexOf(':'); 160 | Preconditions.checkArgument(index > 0 && index < line.length() - 1); 161 | 162 | String host = line.substring(0, index); 163 | int port = Integer.parseInt(line.substring(index + 1)); 164 | builder.add(new ServiceInstance( 165 | new Endpoint(host, port), // endpoint 166 | Collections.emptyMap(), // additional endpoints 167 | Status.ALIVE)); // status 168 | } 169 | return builder.build(); 170 | } 171 | } 172 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/commons/util/TimeUtils.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.commons.util; 18 | 19 | import com.twitter.common.util.Clock; 20 | 21 | import java.text.DateFormat; 22 | import java.text.ParseException; 23 | import java.text.SimpleDateFormat; 24 | import java.util.Date; 25 | import java.util.TimeZone; 26 | import java.util.concurrent.TimeUnit; 27 | 28 | 29 | /** 30 | * General utilities for date/time conversions. 31 | */ 32 | public class TimeUtils { 33 | 34 | public static final int HOURS_OF_ONE_DAY = 24; 35 | public static final int MINUTES_OF_ONE_HOUR = 60; 36 | public static final int MINUTES_OF_ONE_DAY = 60 * 24; 37 | public static final int SECONDS_OF_ONE_HOUR = 3600; 38 | public static final int SECONDS_OF_ONE_DAY = 24 * 3600; 39 | public static final int SECONDS_OF_ONE_WEEK = 7 * 24 * 3600; 40 | public static final int SECONDS_OF_TWO_WEEKS = 14 * 24 * 3600; 41 | public static final int SECONDS_OF_FOUR_WEEKS = 28 * 24 * 3600; 42 | public static final int SECONDS_OF_NINETY_DAYS = 90 * 24 * 3600; 43 | 44 | private static DateFormat sFormatterInSec = createUTCDateFormatter("yyyy-MM-dd HH:mm:ss"); 45 | private static DateFormat sFormatterInMin = createUTCDateFormatter("yyyy-MM-dd HH:mm"); 46 | private static DateFormat sFormatterInDay = createUTCDateFormatter("yyyy-MM-dd"); 47 | 48 | /** 49 | * Creates a DateFormat that assumes the incoming date is in UTC time. 50 | * @param dateFormat - the format string, e.g. "yyyy-MM-dd". 51 | */ 52 | public static DateFormat createUTCDateFormatter(String dateFormat) { 53 | DateFormat formatter = new SimpleDateFormat(dateFormat); 54 | formatter.setTimeZone(TimeZone.getTimeZone("UTC")); 55 | return formatter; 56 | } 57 | 58 | /** 59 | * Get the current timestamp in milliseconds. 60 | */ 61 | public static long getNowTimestampInMilliseconds() { 62 | Date date = new Date(); 63 | return date.getTime(); 64 | } 65 | 66 | /** 67 | * More accurate form of current time in milliseconds, based on System.nanoTime(). 68 | * NOTE: This is not wall clock time, and should only be used for relative time measurement. 69 | */ 70 | public static long millisTime() { 71 | return TimeUnit.NANOSECONDS.toMillis(Clock.SYSTEM_CLOCK.nowNanos()); 72 | } 73 | 74 | /* 75 | * Example input: 2013-06-08 23:59:59 and 2013-06-09 00:00:01 76 | * Example output: 1 77 | */ 78 | public static long getDiffInDays(Date date1, Date date2) { 79 | long ts1 = stringInDaysToTimestampInSeconds(dateToStringInDays(date1)); 80 | long ts2 = stringInDaysToTimestampInSeconds(dateToStringInDays(date2)); 81 | return Math.abs(ts1 - ts2) / SECONDS_OF_ONE_DAY; 82 | } 83 | 84 | /* 85 | * Example input: timestamp representing 2013-06-08 23:59:59 and 2013-06-09 00:00:01 86 | * Example output: 1 87 | */ 88 | public static long getDiffInDays(long ts1, long ts2) { 89 | return getDiffInDays(timestampInSecondsToDate(ts1), timestampInSecondsToDate(ts2)); 90 | } 91 | 92 | /** 93 | * Get the current timestamp in seconds. 94 | */ 95 | public static long getNowTimestampInSeconds() { 96 | return getNowTimestampInMilliseconds() / 1000; 97 | } 98 | 99 | /** 100 | * Convert a date time string (e.g., "2011-06-07 19:12:18") to a Date object. 101 | */ 102 | public static Date stringToDate(String s, DateFormat formatter) { 103 | try { 104 | return formatter.parse(s); 105 | } catch (ParseException e) { 106 | return null; 107 | } 108 | } 109 | 110 | public static long getStartTimestampOfADay(long timestamp) { 111 | return stringInDaysToTimestampInSeconds( 112 | dateToStringInDays(timestampInSecondsToDate(timestamp))); 113 | } 114 | 115 | public static Date stringInSecondsToDate(String s) { 116 | return stringToDate(s, sFormatterInSec); 117 | } 118 | 119 | public static Date timestampInSecondsToDate(long timestamp) { 120 | return new Date(timestamp * 1000); 121 | } 122 | 123 | public static String dateToStringInDays(Date date) { 124 | return sFormatterInDay.format(date); 125 | } 126 | 127 | public static String dateToStringInMinutes(Date date) { 128 | return sFormatterInMin.format(date); 129 | } 130 | 131 | /** 132 | * Convert a date time string (e.g., "2011-06-07 19:12:18") to a timestamp 133 | * in milliseconds (i.e., the number of milliseconds since January 1, 1970, 00:00:00 GMT. 134 | */ 135 | public static long stringToTimestampInMilliseconds(String s, DateFormat formatter) { 136 | Date d = stringToDate(s, formatter); 137 | return d == null ? 0 : d.getTime(); 138 | } 139 | 140 | /** 141 | * Convert a date time string (e.g., "2011-06-07 19:12:18") to a timestamp 142 | * in seconds (i.e., the number of seconds since January 1, 1970, 00:00:00 GMT. 143 | */ 144 | public static long stringToTimestampInSeconds(String s, DateFormat formatter) { 145 | return stringToTimestampInMilliseconds(s, formatter) / 1000; 146 | } 147 | 148 | public static long stringInSecondsToTimestampInSeconds(String s) { 149 | return stringToTimestampInSeconds(s, sFormatterInSec); 150 | } 151 | 152 | public static long stringInDaysToTimestampInSeconds(String s) { 153 | return stringToTimestampInSeconds(s, sFormatterInDay); 154 | } 155 | 156 | public static long stringInMinutesToTimestampInSeconds(String s) { 157 | return stringToTimestampInSeconds(s, sFormatterInMin); 158 | } 159 | 160 | /** 161 | * Calculate a time decay penalty score in (0.3, 1). 162 | * 163 | * The demotion function used here is based on the exponential decay and 164 | * maps the value to interval (0.3, 1). 165 | * 166 | * D(x) = 0.3 + 0.7 * (1 - 0.1)^delta(t) 167 | * 168 | * where 0.3 the lower bound of the penaly score, 0.7 is the initial value 169 | * and 0.1 (10%) is the percent decrease over a day, and delta(t) is the 170 | * days between a pin's creation time and now. 171 | * 172 | * Search "y = 0.3 + 0.7 * (1 - 0.1)^x" at google to visualize the function. 173 | * 174 | * Examples: 175 | * D(0.0) = 1.0 176 | * D(0.5) = 0.9640783086353595 177 | * P(1.0) = 0.9299999999999999 178 | * P(2.0) = 0.8670000000000000 179 | * P(3.0) = 0.810300000000000 180 | * P(4.0) = 0.7592699999999999 181 | * P(5.0) = 0.7133430000000001 182 | * P(10.0) = 0.54407490807 183 | * P(20.0) = 0.38510365821339854 184 | * P(30.0) = 0.32967381079265135 185 | * P(365.0) = 0.300000000000000 186 | * 187 | * Args: 188 | * eventTimestamp: the timestamp when the event took place. 189 | * nowTimestamp: the current timestamp. 190 | * 191 | * Returns: 192 | * The time decay penalty of the event. 193 | */ 194 | public static double calculateTimeDecayPenalty(long eventTimestamp, long nowTimestamp) { 195 | if (nowTimestamp == 0) { 196 | return 1.0; 197 | } 198 | 199 | if (eventTimestamp == 0) { 200 | return 0.3; 201 | } 202 | 203 | // Calculate the days between the event and now. 204 | double days = (nowTimestamp - eventTimestamp) / 86400.0; 205 | return calculateTimeDecayPenalty(0.3, 0.7, 0.9, days); 206 | } 207 | 208 | /** 209 | * Calculate the arbitary time decay: y = base + boost * (param)^days 210 | */ 211 | public static double calculateTimeDecayPenalty(double base, double boost, 212 | double param, double days) { 213 | if (days <= 0.0) { 214 | return base + boost; 215 | } 216 | return base + boost * Math.pow(param, days); 217 | } 218 | 219 | /** 220 | * Return the current wall-clock time in nanoseconds. 221 | */ 222 | public static long getNowTimestampInNanos() { 223 | return System.currentTimeMillis() * 1000 * 1000; 224 | } 225 | } 226 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/example/PinLaterExampleWorker.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.example; 18 | 19 | import com.pinterest.pinlater.client.PinLaterClient; 20 | import com.pinterest.pinlater.commons.config.ConfigFileServerSet; 21 | import com.pinterest.pinlater.commons.util.BytesUtil; 22 | import com.pinterest.pinlater.thrift.PinLaterDequeueMetadata; 23 | import com.pinterest.pinlater.thrift.PinLaterDequeueRequest; 24 | import com.pinterest.pinlater.thrift.PinLaterDequeueResponse; 25 | import com.pinterest.pinlater.thrift.PinLaterJobAckInfo; 26 | import com.pinterest.pinlater.thrift.PinLaterJobAckRequest; 27 | import com.pinterest.pinlater.thrift.RequestContext; 28 | import com.twitter.common.zookeeper.ServerSet; 29 | import com.twitter.util.Function; 30 | import org.slf4j.Logger; 31 | import org.slf4j.LoggerFactory; 32 | import scala.runtime.BoxedUnit; 33 | 34 | import java.net.InetAddress; 35 | import java.net.UnknownHostException; 36 | import java.nio.ByteBuffer; 37 | import java.util.ArrayList; 38 | import java.util.List; 39 | import java.util.Map; 40 | import java.util.concurrent.BlockingQueue; 41 | import java.util.concurrent.ExecutorService; 42 | import java.util.concurrent.Executors; 43 | import java.util.concurrent.LinkedBlockingQueue; 44 | import java.util.concurrent.ScheduledExecutorService; 45 | import java.util.concurrent.TimeUnit; 46 | import java.util.concurrent.atomic.AtomicInteger; 47 | 48 | /** 49 | * An example PinLater worker implementation. 50 | * 51 | * It uses two separate threads for dequeue and ACK, and a thread pool for executing the jobs. 52 | * Completed jobs will be pushed into two queues depending on whether the execution succeeded or 53 | * failed. Note that the dequeue thread can also send ACK along with the dequeue request. The 54 | * worker also implements a linear backoff retry policy, where the retry delay is calculated with 55 | * the number of retry allowed and remained. 56 | * 57 | * File-based serverset is used for service discovery. It uses a local file that stores the 58 | * servers' [HOST_IP]:[PORT] pairs instead of talking to Zookeeper directly. 59 | */ 60 | public class PinLaterExampleWorker { 61 | 62 | private static final int DEQUEUE_BATCH_SIZE = 10; 63 | private static final int NUM_WORKER_THREADS = 10; 64 | private static final int DEQUEUE_INTEVAL_MS = 1000; 65 | private static final int ACK_INTEVAL_MS = 1000; 66 | private static final int PENDING_JOB_LIMIT = 50; 67 | 68 | private static final Logger LOG = LoggerFactory.getLogger(PinLaterExampleWorker.class); 69 | private static final RequestContext REQUEST_CONTEXT; 70 | static { 71 | try { 72 | REQUEST_CONTEXT = new RequestContext( 73 | "pinlaterexampleworker:" + InetAddress.getLocalHost().getHostName()); 74 | } catch (UnknownHostException e) { 75 | LOG.error("Failed to initializer PinLaterExampleWorker", e); 76 | throw new RuntimeException(e); 77 | } 78 | } 79 | 80 | // Max number of pending/running jobs. The worker will stop dequeue when reaches this limit. 81 | private final AtomicInteger numPendingJobs = new AtomicInteger(0); 82 | // Thread pool for executing PinLater jobs. 83 | private final ExecutorService workerExecutor = Executors.newFixedThreadPool(NUM_WORKER_THREADS); 84 | 85 | // Local buffer for succeeded and failed jobs, waiting for ACK 86 | private final BlockingQueue succeededJobQueue = 87 | new LinkedBlockingQueue(); 88 | private final BlockingQueue failedJobQueue = 89 | new LinkedBlockingQueue(); 90 | 91 | private PinLaterClient client; 92 | 93 | public PinLaterExampleWorker() { 94 | String fullServerSetPath = 95 | getClass().getResource("/" + System.getProperty("serverset_path")).getPath(); 96 | ServerSet serverSet = new ConfigFileServerSet(fullServerSetPath); 97 | this.client = new PinLaterClient(serverSet, 10); 98 | 99 | ScheduledExecutorService dequeueAckExecutor = Executors.newScheduledThreadPool(2); 100 | dequeueAckExecutor.scheduleWithFixedDelay( 101 | new DequeueThread(), 0, DEQUEUE_INTEVAL_MS, TimeUnit.MILLISECONDS); 102 | dequeueAckExecutor.scheduleWithFixedDelay( 103 | new AckThread(), 0, ACK_INTEVAL_MS, TimeUnit.MILLISECONDS); 104 | } 105 | 106 | public static void main(String[] args) { 107 | new PinLaterExampleWorker(); 108 | } 109 | 110 | private PinLaterJobAckRequest buildAckRequest() { 111 | List succeededJobs = new ArrayList(); 112 | List failedJobs = new ArrayList(); 113 | succeededJobQueue.drainTo(succeededJobs); 114 | failedJobQueue.drainTo(failedJobs); 115 | if (succeededJobs.size() > 0 || failedJobs.size() > 0) { 116 | LOG.info("ACK {}: {} succeeded, {} failed", PinLaterExampleJob.QUEUE_NAME, 117 | succeededJobs.size(), failedJobs.size()); 118 | PinLaterJobAckRequest ackRequest = 119 | new PinLaterJobAckRequest(PinLaterExampleJob.QUEUE_NAME); 120 | ackRequest.setJobsSucceeded(succeededJobs); 121 | ackRequest.setJobsFailed(failedJobs); 122 | return ackRequest; 123 | } else { 124 | return null; 125 | } 126 | } 127 | 128 | class DequeueThread implements Runnable { 129 | 130 | @Override 131 | public void run() { 132 | if (numPendingJobs.get() > PENDING_JOB_LIMIT) { 133 | return; 134 | } 135 | 136 | PinLaterDequeueRequest dequeueRequest = 137 | new PinLaterDequeueRequest(PinLaterExampleJob.QUEUE_NAME, DEQUEUE_BATCH_SIZE); 138 | 139 | // Ack completed jobs along with dequeue request 140 | PinLaterJobAckRequest ackRequest = buildAckRequest(); 141 | if (ackRequest != null) { 142 | dequeueRequest.setJobAckRequest(ackRequest); 143 | } 144 | 145 | client.getIface().dequeueJobs(REQUEST_CONTEXT, dequeueRequest).onSuccess( 146 | new Function() { 147 | @Override 148 | public BoxedUnit apply(final PinLaterDequeueResponse response) { 149 | LOG.info("DEQUEUE {}: {} jobs, {} jobs pending", 150 | PinLaterExampleJob.QUEUE_NAME, response.getJobsSize(), numPendingJobs.get()); 151 | for (final Map.Entry job : response.getJobs().entrySet()) { 152 | numPendingJobs.incrementAndGet(); 153 | workerExecutor.submit(new Runnable() { 154 | @Override 155 | public void run() { 156 | try { 157 | PinLaterExampleJob.process( 158 | new String(BytesUtil.readBytesFromByteBuffer(job.getValue()))); 159 | succeededJobQueue.add(new PinLaterJobAckInfo(job.getKey())); 160 | } catch (Exception e) { 161 | PinLaterJobAckInfo ackInfo = new PinLaterJobAckInfo(job.getKey()); 162 | 163 | // Append exception message to the custom status 164 | ackInfo.setAppendCustomStatus(e.getMessage()); 165 | 166 | // Retry with linear backoff, e.g. 1s, 2s, 3s ... 167 | PinLaterDequeueMetadata metaData = 168 | response.getJobMetadata().get(job.getKey()); 169 | int attemptsAllowed = metaData.getAttemptsAllowed(); 170 | int attemptsRemained = metaData.getAttemptsRemaining(); 171 | ackInfo.setRetryDelayMillis(1000 * (attemptsAllowed - attemptsRemained)); 172 | 173 | failedJobQueue.add(ackInfo); 174 | } finally { 175 | numPendingJobs.decrementAndGet(); 176 | } 177 | } 178 | }); 179 | } 180 | return BoxedUnit.UNIT; 181 | } 182 | } 183 | ); 184 | } 185 | } 186 | 187 | class AckThread implements Runnable { 188 | 189 | @Override 190 | public void run() { 191 | PinLaterJobAckRequest ackRequest = buildAckRequest(); 192 | if (ackRequest != null) { 193 | client.getIface().ackDequeuedJobs(REQUEST_CONTEXT, ackRequest); 194 | } 195 | } 196 | } 197 | } 198 | -------------------------------------------------------------------------------- /src/test/java/com/pinterest/pinlater/backends/common/BackendQueueMonitorBaseTest.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.common; 18 | 19 | import com.pinterest.pinlater.PinLaterBackendBase; 20 | import com.pinterest.pinlater.thrift.PinLaterDequeueRequest; 21 | import com.pinterest.pinlater.thrift.PinLaterDequeueResponse; 22 | import com.pinterest.pinlater.thrift.PinLaterEnqueueRequest; 23 | import com.pinterest.pinlater.thrift.PinLaterEnqueueResponse; 24 | import com.pinterest.pinlater.thrift.PinLaterJob; 25 | import com.pinterest.pinlater.thrift.PinLaterJobAckInfo; 26 | import com.pinterest.pinlater.thrift.PinLaterJobAckRequest; 27 | import com.pinterest.pinlater.thrift.PinLaterJobInfo; 28 | import com.pinterest.pinlater.thrift.PinLaterJobState; 29 | import com.pinterest.pinlater.thrift.PinLaterLookupJobRequest; 30 | 31 | import com.google.common.collect.Lists; 32 | import org.junit.Assert; 33 | import org.junit.Test; 34 | 35 | import java.nio.ByteBuffer; 36 | import java.util.List; 37 | import java.util.Map; 38 | import java.util.Random; 39 | import java.util.concurrent.TimeUnit; 40 | 41 | public abstract class BackendQueueMonitorBaseTest { 43 | 44 | private static final Random RANDOM = new Random(); 45 | 46 | protected abstract String getQueueName(); 47 | 48 | protected abstract T1 getBackend(); 49 | 50 | /** 51 | * Creates a queue monitor with custom settings. We will invoke this ourselves. 52 | * Note that the backend will create a queue monitor on a separate thread too, 53 | * but the default configuration means it won't run for the duration of this test, 54 | * so we are fine. 55 | */ 56 | protected abstract T2 createQueueMonitor(long jobClaimedTimeoutMillis, 57 | long jobSucceededGCTimeoutMillis, 58 | long jobFailedGCTimeoutMillis); 59 | 60 | @Test 61 | public void testJobGC() throws InterruptedException { 62 | List jobDescriptors = Lists.newArrayList(); 63 | 64 | // Enqueue 100 jobs with no retries and at random priorities. 65 | // We intentionally don't use a batch request, to ensure distribution across shards. 66 | for (int i = 0; i < 100; i++) { 67 | PinLaterEnqueueRequest enqueueRequest = new PinLaterEnqueueRequest(); 68 | enqueueRequest.setQueueName(getQueueName()); 69 | PinLaterJob job = new PinLaterJob(ByteBuffer.wrap(new String("job_body_" + i).getBytes())); 70 | job.setNumAttemptsAllowed(1); 71 | job.setPriority((byte) (RANDOM.nextInt(3) + 1)); 72 | enqueueRequest.addToJobs(job); 73 | PinLaterEnqueueResponse enqueueResponse = getBackend().enqueueJobs(enqueueRequest).get(); 74 | Assert.assertEquals(1, enqueueResponse.getJobDescriptorsSize()); 75 | jobDescriptors.add(enqueueResponse.getJobDescriptors().get(0)); 76 | } 77 | 78 | // Lookup the jobs and validate state. 79 | PinLaterLookupJobRequest lookupJobRequest = new PinLaterLookupJobRequest(); 80 | for (String jobDesc : jobDescriptors) { 81 | lookupJobRequest.addToJobDescriptors(jobDesc); 82 | } 83 | Map jobInfoMap = getBackend().lookupJobs(lookupJobRequest).get(); 84 | Assert.assertEquals(100, jobInfoMap.size()); 85 | for (PinLaterJobInfo jobInfo : jobInfoMap.values()) { 86 | Assert.assertEquals(PinLaterJobState.PENDING, jobInfo.getJobState()); 87 | } 88 | 89 | // Dequeue all 100. 90 | PinLaterDequeueRequest dequeueRequest = new PinLaterDequeueRequest(getQueueName(), 100); 91 | PinLaterDequeueResponse 92 | dequeueResponse = 93 | getBackend().dequeueJobs("test", dequeueRequest).get(); 94 | Assert.assertEquals(100, PinLaterTestUtils.getJobCount(getBackend(), getQueueName(), 95 | PinLaterJobState.IN_PROGRESS)); 96 | 97 | // Lookup the jobs and validate state. 98 | jobInfoMap = getBackend().lookupJobs(lookupJobRequest).get(); 99 | Assert.assertEquals(100, jobInfoMap.size()); 100 | for (PinLaterJobInfo jobInfo : jobInfoMap.values()) { 101 | Assert.assertEquals(PinLaterJobState.IN_PROGRESS, jobInfo.getJobState()); 102 | } 103 | 104 | // Ack 50 jobs as succeeded, 50 as failed. 105 | PinLaterJobAckRequest jobAckRequest = new PinLaterJobAckRequest(getQueueName()); 106 | boolean alternate = false; 107 | for (String jobDesc : jobDescriptors) { 108 | if (alternate) { 109 | jobAckRequest.addToJobsSucceeded(new PinLaterJobAckInfo(jobDesc)); 110 | } else { 111 | jobAckRequest.addToJobsFailed(new PinLaterJobAckInfo(jobDesc)); 112 | } 113 | alternate = !alternate; 114 | } 115 | getBackend().ackDequeuedJobs(jobAckRequest).get(); 116 | Assert.assertEquals(50, PinLaterTestUtils.getJobCount(getBackend(), getQueueName(), 117 | PinLaterJobState.SUCCEEDED)); 118 | Assert.assertEquals(50, PinLaterTestUtils.getJobCount(getBackend(), getQueueName(), 119 | PinLaterJobState.FAILED)); 120 | 121 | // Backend timestamps are in second resolution, so one second is the minimum we'll need 122 | // to sleep to ensure this test works. 123 | Thread.sleep(TimeUnit.SECONDS.toMillis(1)); 124 | 125 | // Run queue monitor configured with low succeeded GC timeout. 126 | createQueueMonitor(TimeUnit.HOURS.toMillis(1), 1, TimeUnit.HOURS.toMillis(1)).run(); 127 | 128 | // All succeeded jobs should have been GC'ed, but not failed jobs. 129 | Assert.assertEquals(0, PinLaterTestUtils.getJobCount(getBackend(), getQueueName(), 130 | PinLaterJobState.SUCCEEDED)); 131 | Assert.assertEquals(50, PinLaterTestUtils.getJobCount(getBackend(), getQueueName(), 132 | PinLaterJobState.FAILED)); 133 | 134 | // Run queue monitor configured with low failed GC timeout. 135 | createQueueMonitor(TimeUnit.HOURS.toMillis(1), TimeUnit.HOURS.toMillis(1), 1).run(); 136 | 137 | // Now failed jobs should also be GC'ed. 138 | Assert.assertEquals(0, PinLaterTestUtils.getJobCount(getBackend(), getQueueName(), 139 | PinLaterJobState.FAILED)); 140 | 141 | // Looking up the jobs should return an empty map. 142 | jobInfoMap = getBackend().lookupJobs(lookupJobRequest).get(); 143 | Assert.assertEquals(0, jobInfoMap.size()); 144 | } 145 | 146 | @Test 147 | public void testJobClaimedTimeout() throws InterruptedException { 148 | // Enqueue 100 jobs at random priorities, 50 of them with no retries, and 50 with 1 retry. 149 | // We intentionally don't use a batch request, to ensure distribution across shards. 150 | for (int i = 0; i < 100; i++) { 151 | PinLaterEnqueueRequest enqueueRequest = new PinLaterEnqueueRequest(); 152 | enqueueRequest.setQueueName(getQueueName()); 153 | PinLaterJob job = new PinLaterJob(ByteBuffer.wrap(new String("job_body_" + i).getBytes())); 154 | job.setNumAttemptsAllowed(i < 50 ? 1 : 2); 155 | job.setPriority((byte) (RANDOM.nextInt(3) + 1)); 156 | enqueueRequest.addToJobs(job); 157 | PinLaterEnqueueResponse enqueueResponse = getBackend().enqueueJobs(enqueueRequest).get(); 158 | Assert.assertEquals(1, enqueueResponse.getJobDescriptorsSize()); 159 | } 160 | 161 | // Dequeue all 100. 162 | PinLaterDequeueRequest dequeueRequest = new PinLaterDequeueRequest(getQueueName(), 100); 163 | PinLaterDequeueResponse 164 | dequeueResponse = 165 | getBackend().dequeueJobs("test", dequeueRequest).get(); 166 | Assert.assertEquals(100, PinLaterTestUtils.getJobCount(getBackend(), getQueueName(), 167 | PinLaterJobState.IN_PROGRESS)); 168 | 169 | // Backend timestamps are in second resolution, so one second is the minimum we'll need 170 | // to sleep to ensure this test works. 171 | Thread.sleep(TimeUnit.SECONDS.toMillis(1)); 172 | 173 | // Run queue monitor configured with low job claimed timeout. 174 | createQueueMonitor(1, TimeUnit.HOURS.toMillis(1), TimeUnit.HOURS.toMillis(1)).run(); 175 | 176 | // 50 jobs should go to PENDING, remaining 50 should go to FAILED. 177 | Assert.assertEquals(50, PinLaterTestUtils.getJobCount(getBackend(), getQueueName(), 178 | PinLaterJobState.PENDING)); 179 | Assert.assertEquals(50, PinLaterTestUtils.getJobCount(getBackend(), getQueueName(), 180 | PinLaterJobState.FAILED)); 181 | } 182 | } 183 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/redis/RedisQueueMonitor.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.redis; 18 | 19 | import com.pinterest.pinlater.backends.common.BackendQueueMonitorBase; 20 | import com.pinterest.pinlater.commons.healthcheck.HealthChecker; 21 | import com.pinterest.pinlater.thrift.PinLaterJobState; 22 | 23 | import com.google.common.annotations.VisibleForTesting; 24 | import com.google.common.base.Preconditions; 25 | import com.google.common.collect.ImmutableMap; 26 | import com.google.common.collect.Lists; 27 | import com.twitter.ostrich.stats.Stats; 28 | import com.twitter.util.Function; 29 | import org.apache.commons.configuration.PropertiesConfiguration; 30 | import org.slf4j.Logger; 31 | import org.slf4j.LoggerFactory; 32 | import redis.clients.jedis.Jedis; 33 | 34 | import java.util.List; 35 | import java.util.Map; 36 | import java.util.Set; 37 | 38 | /** 39 | * Implements a scheduled task used by the PinLaterRedisBackend for various types of 40 | * job queue cleanup, including ACK timeouts and GC'ing finished jobs. 41 | */ 42 | public class RedisQueueMonitor extends BackendQueueMonitorBase { 43 | 44 | private static final Logger LOG = LoggerFactory.getLogger(RedisQueueMonitor.class); 45 | 46 | private int logCount; 47 | private int numTimeoutDone; 48 | private int numTimeoutRetry; 49 | private long numSucceededGC; 50 | private long numFailedGC; 51 | private int numTimeoutEvict; 52 | private final HealthChecker healthChecker; 53 | 54 | public RedisQueueMonitor(ImmutableMap shardMap, 55 | PropertiesConfiguration configuration, 56 | HealthChecker healthChecker) { 57 | super(shardMap, configuration); 58 | this.healthChecker = Preconditions.checkNotNull(healthChecker); 59 | } 60 | 61 | @VisibleForTesting 62 | public RedisQueueMonitor(ImmutableMap shardMap, 63 | int updateMaxSize, int maxAutoRetries, int logInterval, 64 | long jobClaimedTimeoutMillis, long jobSucceededGCTimeoutMillis, 65 | long jobFailedGCTimeoutMillis, int numPriorityLevels, 66 | HealthChecker healthChecker) { 67 | super(shardMap, updateMaxSize, maxAutoRetries, logInterval, jobClaimedTimeoutMillis, 68 | jobSucceededGCTimeoutMillis, jobFailedGCTimeoutMillis, numPriorityLevels); 69 | this.healthChecker = Preconditions.checkNotNull(healthChecker); 70 | } 71 | 72 | @Override 73 | protected void jobMonitorImpl(long runStartMillis, 74 | final Map.Entry shard, 75 | int numAutoRetries) { 76 | // Skip the shard if it is unhealthy. 77 | if (!healthChecker.isServerLive( 78 | shard.getValue().getHost(), shard.getValue().getPort())) { 79 | LOG.warn(String.format("Skipped monitoring shard %s because it is unhealthy.", 80 | shard.getKey())); 81 | return; 82 | } 83 | 84 | final double runStartTimeSeconds = runStartMillis / 1000.0; 85 | final double succeededGCTimeSeconds = (runStartMillis - getJobSucceededGCTimeoutMillis()) 86 | / 1000.0; 87 | final double failedGCTimeSeconds = (runStartMillis - getJobFailedGCTimeoutMillis()) / 1000.0; 88 | final double timeoutTimeSeconds = (runStartMillis - getJobClaimedTimeoutMillis()) / 1000.0; 89 | 90 | try { 91 | RedisUtils.executeWithConnection( 92 | shard.getValue().getMonitorRedisPool(), 93 | new Function() { 94 | @Override 95 | public Void apply(Jedis conn) { 96 | Set queueNames = RedisBackendUtils.getQueueNames(conn, shard.getKey()); 97 | for (String queueName : queueNames) { 98 | for (int priority = 1; 99 | priority <= numPriorityLevels; 100 | priority++) { 101 | String pendingQueueRedisKey = RedisBackendUtils.constructQueueRedisKey( 102 | queueName, shard.getKey(), priority, PinLaterJobState.PENDING); 103 | String inProgressQueueRedisKey = RedisBackendUtils.constructQueueRedisKey( 104 | queueName, shard.getKey(), priority, PinLaterJobState.IN_PROGRESS); 105 | String succeededQueueRedisKey = RedisBackendUtils.constructQueueRedisKey( 106 | queueName, shard.getKey(), priority, PinLaterJobState.SUCCEEDED); 107 | String failedQueueRedisKey = RedisBackendUtils.constructQueueRedisKey( 108 | queueName, shard.getKey(), priority, PinLaterJobState.FAILED); 109 | String hashRedisKeyPrefix = RedisBackendUtils.constructHashRedisKeyPrefix( 110 | queueName, shard.getKey()); 111 | 112 | // Handle timed out jobs. 113 | List keys = Lists.newArrayList( 114 | inProgressQueueRedisKey, 115 | hashRedisKeyPrefix, 116 | pendingQueueRedisKey, 117 | failedQueueRedisKey); 118 | List argv = Lists.newArrayList( 119 | String.valueOf(timeoutTimeSeconds), 120 | String.valueOf(getUpdateMaxSize()), 121 | String.valueOf(runStartTimeSeconds)); 122 | Object nums = conn.eval( 123 | RedisLuaScripts.MONITOR_TIMEOUT_UPDATE, keys, argv); 124 | List tmp = (List) nums; 125 | numTimeoutDone += Integer.valueOf((String) tmp.get(0)); 126 | numTimeoutRetry += Integer.valueOf((String) tmp.get(1)); 127 | numTimeoutEvict += Integer.valueOf((String) tmp.get(2)); 128 | 129 | // Succeeded job GC. 130 | keys = Lists.newArrayList(succeededQueueRedisKey, hashRedisKeyPrefix); 131 | argv = Lists.newArrayList(String.valueOf(succeededGCTimeSeconds), 132 | String.valueOf(getUpdateMaxSize())); 133 | numSucceededGC += (Long) conn.eval( 134 | RedisLuaScripts.MONITOR_GC_DONE_JOBS, keys, argv); 135 | 136 | // Failed job GC. 137 | keys = Lists.newArrayList(failedQueueRedisKey, hashRedisKeyPrefix); 138 | argv = Lists.newArrayList(String.valueOf(failedGCTimeSeconds), 139 | String.valueOf(getUpdateMaxSize())); 140 | numFailedGC += (Long) conn.eval( 141 | RedisLuaScripts.MONITOR_GC_DONE_JOBS, keys, argv); 142 | 143 | logCount++; 144 | if (logCount % getLogInterval() == 0) { 145 | LOG.info(String.format( 146 | "JobQueueMonitor: " 147 | + "Shard: %s Queue: %s Priority: %d Timeout Done: %d Timeout Retry: %d " 148 | + "Succeeded GC: %d Failed GC: %d", 149 | shard.getKey(), queueName, priority, numTimeoutDone, numTimeoutRetry, 150 | numSucceededGC, numFailedGC)); 151 | if (numTimeoutEvict != 0) { 152 | LOG.error(String.format( 153 | "JobQueueMonitor: Shard: %s Queue: %s Priority: %d Timeout Evict: %d", 154 | shard.getKey(), queueName, priority, numTimeoutEvict)); 155 | } 156 | Stats.incr(queueName + "_timeout_done", numTimeoutDone); 157 | Stats.incr(queueName + "_timeout_retry", numTimeoutRetry); 158 | Stats.incr(queueName + "_timeout_evict", numTimeoutEvict); 159 | Stats.incr(queueName + "_succeeded_gc", (int) numSucceededGC); 160 | Stats.incr(queueName + "_failed_gc", (int) numFailedGC); 161 | logCount = 0; 162 | numTimeoutDone = 0; 163 | numTimeoutRetry = 0; 164 | numSucceededGC = 0; 165 | numFailedGC = 0; 166 | numTimeoutEvict = 0; 167 | } 168 | } 169 | } 170 | return null; 171 | } 172 | }); 173 | } catch (Exception e) { 174 | Stats.incr("PinLater.RedisQueueMonitor.errors." + e.getClass().getSimpleName()); 175 | LOG.error("Exception in JobQueueMonitor task", e); 176 | } 177 | } 178 | } 179 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/backends/mysql/MySQLQueries.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater.backends.mysql; 18 | 19 | import com.google.common.annotations.VisibleForTesting; 20 | 21 | /** 22 | * Encapsulates MySQL queries used by the PinLaterMySQLBackend and related classes. 23 | */ 24 | public final class MySQLQueries { 25 | 26 | private MySQLQueries() {} 27 | 28 | @VisibleForTesting 29 | public static final int CUSTOM_STATUS_SIZE_BYTES = 5000; 30 | 31 | private static final String TRUNCATED_CUSTOM_STATUS = 32 | "SUBSTRING(?, 1, " + CUSTOM_STATUS_SIZE_BYTES + ")"; 33 | 34 | private static final String TRUNCATED_CONCAT_CUSTOM_STATUS = 35 | "SUBSTRING(CONCAT(custom_status, ?), 1, " + CUSTOM_STATUS_SIZE_BYTES + ")"; 36 | 37 | private static final String TRUNCATED_PREPEND_CUSTOM_STATUS = 38 | "SUBSTRING(CONCAT(?, custom_status), 1, " + CUSTOM_STATUS_SIZE_BYTES + ")"; 39 | 40 | public static final String BODY_REGEX_CLAUSE = "AND body REGEXP '%s'"; 41 | 42 | public static final String CREATE_DATABASE = 43 | "CREATE DATABASE %s"; 44 | 45 | public static final String CREATE_JOBS_TABLE = 46 | "CREATE TABLE %s" 47 | + "( local_id BIGINT UNSIGNED NOT NULL AUTO_INCREMENT" 48 | + ", state TINYINT NOT NULL" 49 | + ", claim_descriptor VARCHAR(200)" 50 | + ", attempts_allowed INT" 51 | + ", attempts_remaining INT NOT NULL" 52 | + ", custom_status VARCHAR(" + CUSTOM_STATUS_SIZE_BYTES + ")" 53 | + ", updated_at TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP" 54 | + ", created_at TIMESTAMP NOT NULL" 55 | + ", run_after TIMESTAMP NOT NULL" 56 | + ", body BLOB NOT NULL" 57 | + ", PRIMARY KEY (local_id), INDEX (state, run_after), INDEX (claim_descriptor)" 58 | + " ) ENGINE=INNODB"; 59 | 60 | public static final String ENQUEUE_INSERT = 61 | "INSERT INTO %s" 62 | + " (state, attempts_allowed, attempts_remaining, custom_status, created_at, run_after," 63 | + " body)" 64 | + " VALUES (?, ?, ?, " + TRUNCATED_CUSTOM_STATUS + ", ?, ?, ?)"; 65 | 66 | public static final String DEQUEUE_UPDATE = 67 | "UPDATE %s" 68 | + " SET claim_descriptor = ?, state = ?" 69 | + " WHERE state = ? AND run_after <= NOW()" 70 | + " LIMIT ?"; 71 | 72 | public static final String DEQUEUE_SELECT = 73 | "SELECT local_id, attempts_allowed, attempts_remaining, updated_at, created_at, " 74 | + " body FROM %s" 75 | + " WHERE claim_descriptor = ?"; 76 | 77 | public static final String DEQUEUE_DRY_RUN_SELECT = 78 | "SELECT local_id, attempts_allowed, attempts_remaining, updated_at, created_at, " 79 | + " body FROM %s" 80 | + " WHERE state = ? AND run_after <= NOW()" 81 | + " LIMIT ?"; 82 | 83 | public static final String LOOKUP_JOB = 84 | "SELECT local_id, state, attempts_allowed, attempts_remaining, created_at, run_after, " 85 | + " updated_at, claim_descriptor, custom_status FROM %s" 86 | + " WHERE local_id = ?"; 87 | 88 | public static final String LOOKUP_JOB_WITH_BODY = 89 | "SELECT local_id, state, attempts_allowed, attempts_remaining, created_at, run_after, " 90 | + " updated_at, claim_descriptor, custom_status, body FROM %s" 91 | + " WHERE local_id = ?"; 92 | 93 | // Note: a nested query is used here in order to allow for us to cap the count at a certain 94 | // threshold. The reason for this is executing count queries for millions of rows will take on 95 | // the order of seconds, which is too slow. For reference, a count query capped with a limit of 96 | // 100k runs in about ~0.1 s. 97 | public static final String COUNT_CURRENT_JOBS_BY_STATE_PRIORITY = 98 | "SELECT COUNT(*) FROM (" 99 | + " SELECT 1 FROM %s" 100 | + " WHERE state = ? AND run_after <= NOW() %s" 101 | + " LIMIT ?) as job_info"; 102 | 103 | public static final String COUNT_FUTURE_JOBS_BY_STATE_PRIORITY = 104 | "SELECT COUNT(*) FROM (" 105 | + " SELECT 1 FROM %s" 106 | + " WHERE state = ? AND run_after > NOW() %s" 107 | + " LIMIT ?) as job_info"; 108 | 109 | public static final String SCAN_CURRENT_JOBS = 110 | "SELECT local_id, claim_descriptor, attempts_allowed, attempts_remaining, custom_status," 111 | + " created_at, run_after, updated_at FROM %s" 112 | + " WHERE state = ? AND run_after <= NOW() %s" 113 | + " ORDER BY run_after DESC" 114 | + " LIMIT ?"; 115 | 116 | public static final String SCAN_FUTURE_JOBS = 117 | "SELECT local_id, claim_descriptor, attempts_allowed, attempts_remaining, custom_status," 118 | + " created_at, run_after, updated_at FROM %s" 119 | + " WHERE state = ? AND run_after > NOW() %s" 120 | + " ORDER BY run_after DESC" 121 | + " LIMIT ?"; 122 | 123 | public static final String RETRY_FAILED_JOBS = 124 | "UPDATE %s" 125 | + " SET state = ?, attempts_remaining = ?, run_after = ?" 126 | + " WHERE state = ? " 127 | + " LIMIT ?"; 128 | 129 | public static final String DELETE_JOBS = 130 | "DELETE FROM %s" 131 | + " WHERE state = ? %s" 132 | + " LIMIT ?"; 133 | 134 | public static final String ACK_SUCCEEDED_UPDATE = 135 | "UPDATE %s" 136 | + " SET state = ?, " 137 | + " custom_status = " + TRUNCATED_CONCAT_CUSTOM_STATUS 138 | + " WHERE local_id = ?"; 139 | 140 | public static final String ACK_FAILED_DONE_UPDATE = 141 | "UPDATE %s" 142 | + " SET state = ?, " 143 | + " custom_status = " + TRUNCATED_CONCAT_CUSTOM_STATUS + "," 144 | + " attempts_remaining = 0" 145 | + " WHERE local_id = ? AND state = ? AND attempts_remaining = 1"; 146 | 147 | public static final String ACK_FAILED_RETRY_UPDATE = 148 | "UPDATE %s" 149 | + " SET state = ?, " 150 | + " custom_status = " + TRUNCATED_CONCAT_CUSTOM_STATUS + "," 151 | + " run_after = ?," 152 | + " claim_descriptor = NULL," 153 | + " attempts_remaining = attempts_remaining - 1" 154 | + " WHERE local_id = ? AND state = ? AND attempts_remaining > 1"; 155 | 156 | public static final String CHECKPOINT_JOB_HEADER = 157 | "UPDATE %s" 158 | + " SET state = ?" 159 | + ", run_after = ?"; 160 | 161 | public static final String CHECKPOINT_JOB_SET_BODY = 162 | ", body = ?"; 163 | 164 | public static final String CHECKPOINT_JOB_RESET_ATTEMPTS = 165 | ", attempts_allowed = ?" 166 | + ", attempts_remaining = ?"; 167 | 168 | public static final String CHECKPOINT_JOB_APPEND_CUSTOM_STATUS = 169 | ", custom_status = " + TRUNCATED_PREPEND_CUSTOM_STATUS; 170 | 171 | public static final String CHECKPOINT_JOB_RESET_CLAIM_DESCRIPTOR = 172 | ", claim_descriptor = NULL"; 173 | 174 | // We match the claim descriptor when processing a checkpoint request because there can be jobs 175 | // running on workers for a long time that incur a server-side timeout but actually end up 176 | // finishing later on and try to checkpoint even though a second worker is already working 177 | // on the job. For these cases, we would like to make the stale checkpoint request a no-op. 178 | public static final String CHECKPOINT_JOB_FOOTER = 179 | " WHERE local_id = ? AND state = ? AND claim_descriptor LIKE ?"; 180 | 181 | public static final String MONITOR_TIMEOUT_DONE_UPDATE = 182 | "UPDATE %s" 183 | + " SET state = ?, attempts_remaining = 0" 184 | + " WHERE state = ? AND updated_at < ? AND attempts_remaining = 1 LIMIT ?"; 185 | 186 | public static final String MONITOR_TIMEOUT_RETRY_UPDATE = 187 | "UPDATE %s" 188 | + " SET state = ?, claim_descriptor = NULL, attempts_remaining = attempts_remaining - 1" 189 | + " WHERE state = ? AND updated_at < ? AND attempts_remaining > 1 LIMIT ?"; 190 | 191 | // Note: we intentionally use run_after instead of the more appropriate updated_at here 192 | // for query efficiency. In practice, given that the GC time period is long, these should 193 | // be roughly equivalent. 194 | public static final String MONITOR_GC_DONE_JOBS = 195 | "DELETE FROM %s" 196 | + " WHERE state = ? AND run_after < ? LIMIT ?"; 197 | 198 | public static final String DROP_DATABASE = 199 | "DROP DATABASE IF EXISTS %s"; 200 | } 201 | -------------------------------------------------------------------------------- /src/main/java/com/pinterest/pinlater/PinLaterQueueConfig.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.pinterest.pinlater; 18 | 19 | import com.pinterest.pinlater.commons.config.ConfigFileServerSet; 20 | import com.pinterest.pinlater.commons.config.ConfigFileWatcher; 21 | 22 | import com.google.common.annotations.VisibleForTesting; 23 | import com.google.common.base.Preconditions; 24 | import com.google.common.collect.ImmutableMap; 25 | import com.google.common.collect.ImmutableSet; 26 | import com.twitter.common.base.MorePreconditions; 27 | import com.twitter.common.net.pool.DynamicHostSet; 28 | import com.twitter.common.zookeeper.ServerSet; 29 | import com.twitter.thrift.ServiceInstance; 30 | import com.twitter.util.ExceptionalFunction; 31 | import org.apache.commons.configuration.PropertiesConfiguration; 32 | import org.codehaus.jackson.annotate.JsonIgnoreProperties; 33 | import org.codehaus.jackson.map.ObjectMapper; 34 | import org.slf4j.Logger; 35 | import org.slf4j.LoggerFactory; 36 | 37 | import java.io.IOException; 38 | import java.util.List; 39 | import java.util.concurrent.atomic.AtomicInteger; 40 | import java.util.concurrent.atomic.AtomicReference; 41 | 42 | /** 43 | * Class that keeps track of per-queue dynamic configuration for PinLater. 44 | */ 45 | public class PinLaterQueueConfig { 46 | 47 | private static final Logger LOG = LoggerFactory.getLogger(PinLaterQueueConfig.class); 48 | 49 | private final AtomicReference queueConfigSchemaRef = 50 | new AtomicReference(); 51 | private final AtomicInteger numPinLaterServers = new AtomicInteger(1); 52 | private final AtomicReference> queueRateLimitMapRef = 53 | new AtomicReference>(); 54 | 55 | private final String queueConfigFilePath; 56 | private final String pinlaterServerSetPath; 57 | private final boolean pinlaterServerSetEnabled; 58 | 59 | public PinLaterQueueConfig(PropertiesConfiguration configuration) { 60 | this(configuration.getString("QUEUE_CONFIG_FILE_PATH"), 61 | configuration.getString("SERVER_SET_PATH"), 62 | configuration.getBoolean("SERVER_SET_ENABLED")); 63 | } 64 | 65 | @VisibleForTesting 66 | public PinLaterQueueConfig(String queueConfigFilePath, 67 | String pinlaterServerSetPath, 68 | boolean pinlaterServerSetEnabled) { 69 | this.queueConfigFilePath = queueConfigFilePath; 70 | this.pinlaterServerSetPath = pinlaterServerSetPath; 71 | this.pinlaterServerSetEnabled = pinlaterServerSetEnabled; 72 | } 73 | 74 | 75 | public void initialize() throws Exception { 76 | // Check if use of serverset is enabled, and if so, register a change monitor so we 77 | // can find out how many PinLater servers are active. We use this to compute the 78 | // per-server rate limit. 79 | if (pinlaterServerSetEnabled) { 80 | MorePreconditions.checkNotBlank(pinlaterServerSetPath); 81 | LOG.info("Monitoring pinlater serverset: {}", pinlaterServerSetPath); 82 | String fullServerSetPath = getClass().getResource("/" + pinlaterServerSetPath).getPath(); 83 | ServerSet serverSet = new ConfigFileServerSet(fullServerSetPath); 84 | serverSet.monitor(new DynamicHostSet.HostChangeMonitor() { 85 | @Override 86 | public void onChange(ImmutableSet hostSet) { 87 | int oldSize = numPinLaterServers.get(); 88 | int newSize = hostSet.size(); 89 | if (newSize == 0) { 90 | LOG.error("PinLater serverset is empty, ignoring and keeping old size: {}", oldSize); 91 | return; 92 | } 93 | if (oldSize == newSize) { 94 | LOG.info("PinLater serverset update, size unchanged: {}", oldSize); 95 | return; 96 | } 97 | 98 | LOG.info("PinLater serverset update, old size: {}, new size: {}", oldSize, newSize); 99 | numPinLaterServers.set(newSize); 100 | rebuild(); 101 | } 102 | }); 103 | } else { 104 | LOG.info("PinLater server set is disabled; rate limits will be applied per server."); 105 | } 106 | 107 | if (queueConfigFilePath == null || queueConfigFilePath.isEmpty()) { 108 | LOG.info("Queue config zookeeper path not specified, using defaults."); 109 | return; 110 | } 111 | 112 | LOG.info("Registering watch on queue config: {}", queueConfigFilePath); 113 | String fullQueueConfigFilePath = getClass().getResource("/" + queueConfigFilePath).getPath(); 114 | ConfigFileWatcher.defaultInstance().addWatch( 115 | fullQueueConfigFilePath, new ExceptionalFunction() { 116 | @Override 117 | public Void applyE(byte[] bytes) throws Exception { 118 | QueueConfigSchema queueConfigSchema = QueueConfigSchema.load(bytes); 119 | LOG.info("Queue config update, new value: {}", queueConfigSchema); 120 | queueConfigSchemaRef.set(queueConfigSchema); 121 | rebuild(); 122 | return null; 123 | } 124 | }); 125 | } 126 | 127 | /** 128 | * Determines whether a dequeue request should be allowed. 129 | * 130 | * @param queueName name of the queue. 131 | * @param numJobs number of jobs intended to be dequeued. 132 | * @return whether to allow the request. 133 | */ 134 | public boolean allowDequeue(String queueName, int numJobs) { 135 | MorePreconditions.checkNotBlank(queueName); 136 | Preconditions.checkArgument(numJobs > 0); 137 | ImmutableMap queueRateLimitMap = queueRateLimitMapRef.get(); 138 | if (queueRateLimitMap != null && queueRateLimitMap.containsKey(queueName)) { 139 | return queueRateLimitMap.get(queueName).allowDequeue(numJobs); 140 | } else { 141 | // No rate limit specified for this queue, so always allow. 142 | return true; 143 | } 144 | } 145 | 146 | @VisibleForTesting 147 | double getDequeueRate(String queueName) { 148 | MorePreconditions.checkNotBlank(queueName); 149 | ImmutableMap queueRateLimitMap = queueRateLimitMapRef.get(); 150 | if (queueRateLimitMap != null && queueRateLimitMap.containsKey(queueName)) { 151 | return queueRateLimitMap.get(queueName).getRate(); 152 | } else { 153 | // No rate limit specified for this queue. 154 | return Double.MAX_VALUE; 155 | } 156 | } 157 | 158 | /** 159 | * Rebuilds the per-queue configuration. This method is thread-safe. 160 | */ 161 | @VisibleForTesting 162 | synchronized void rebuild() { 163 | QueueConfigSchema queueConfigSchema = queueConfigSchemaRef.get(); 164 | int numServers = numPinLaterServers.get(); 165 | 166 | if (queueConfigSchema == null || queueConfigSchema.queues == null 167 | || queueConfigSchema.queues.isEmpty()) { 168 | return; 169 | } 170 | 171 | ImmutableMap.Builder builder = 172 | new ImmutableMap.Builder(); 173 | for (QueueConfigSchema.Queue queue : queueConfigSchema.queues) { 174 | builder.put(queue.name, 175 | QueueRateLimiter.create(queue.queueConfig.maxJobsPerSecond / numServers)); 176 | } 177 | 178 | queueRateLimitMapRef.set(builder.build()); 179 | } 180 | 181 | @VisibleForTesting 182 | AtomicReference getQueueConfigSchemaRef() { 183 | return queueConfigSchemaRef; 184 | } 185 | 186 | @VisibleForTesting 187 | AtomicInteger getNumPinLaterServers() { 188 | return numPinLaterServers; 189 | } 190 | 191 | /** 192 | * Defines the queue configuration json schema. 193 | */ 194 | public static class QueueConfigSchema { 195 | 196 | private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); 197 | 198 | public List queues; 199 | 200 | public static class Queue { 201 | 202 | public String name; 203 | public QueueConfig queueConfig; 204 | } 205 | 206 | @JsonIgnoreProperties(ignoreUnknown = true) 207 | public static class QueueConfig { 208 | 209 | public double maxJobsPerSecond; 210 | } 211 | 212 | public static QueueConfigSchema load(byte[] bytes) throws IOException { 213 | return QueueConfigSchema.OBJECT_MAPPER.readValue(bytes, QueueConfigSchema.class); 214 | } 215 | 216 | @Override 217 | public String toString() { 218 | StringBuilder out = new StringBuilder(); 219 | out.append("\nQueue Config:"); 220 | for (Queue q : queues) { 221 | out.append("\nQueue: ").append(q.name); 222 | out.append(" maxJobsPerSecond: ").append(q.queueConfig.maxJobsPerSecond); 223 | } 224 | return out.toString(); 225 | } 226 | } 227 | } 228 | --------------------------------------------------------------------------------