Java tutorial
/* * Druid - a distributed column store. * Copyright 2012 - 2015 Metamarkets Group Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package io.druid.metadata; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningScheduledExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import com.metamx.common.MapUtils; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.emitter.EmittingLogger; import io.druid.client.DruidDataSource; import io.druid.concurrent.Execs; import io.druid.guice.ManageLifecycle; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; import org.skife.jdbi.v2.Batch; import org.skife.jdbi.v2.FoldController; import org.skife.jdbi.v2.Folder3; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.IDBI; import org.skife.jdbi.v2.StatementContext; import org.skife.jdbi.v2.tweak.HandleCallback; import org.skife.jdbi.v2.tweak.ResultSetMapper; import org.skife.jdbi.v2.util.ByteArrayMapper; import java.io.IOException; import java.sql.ResultSet; import java.sql.SQLException; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; /** */ @ManageLifecycle public class SQLMetadataSegmentManager implements MetadataSegmentManager { private static final EmittingLogger log = new EmittingLogger(SQLMetadataSegmentManager.class); private final Object lock = new Object(); private final ObjectMapper jsonMapper; private final Supplier<MetadataSegmentManagerConfig> config; private final Supplier<MetadataStorageTablesConfig> dbTables; private final AtomicReference<ConcurrentHashMap<String, DruidDataSource>> dataSources; private final IDBI dbi; private volatile ListeningScheduledExecutorService exec = null; private volatile ListenableFuture<?> future = null; private volatile boolean started = false; @Inject public SQLMetadataSegmentManager(ObjectMapper jsonMapper, Supplier<MetadataSegmentManagerConfig> config, Supplier<MetadataStorageTablesConfig> dbTables, SQLMetadataConnector connector) { this.jsonMapper = jsonMapper; this.config = config; this.dbTables = dbTables; this.dataSources = new AtomicReference<>(new ConcurrentHashMap<String, DruidDataSource>()); this.dbi = connector.getDBI(); } @LifecycleStart public void start() { synchronized (lock) { if (started) { return; } exec = MoreExecutors .listeningDecorator(Execs.scheduledSingleThreaded("DatabaseSegmentManager-Exec--%d")); final Duration delay = config.get().getPollDuration().toStandardDuration(); future = exec.scheduleWithFixedDelay(new Runnable() { @Override public void run() { try { poll(); } catch (Exception e) { log.makeAlert(e, "uncaught exception in segment manager polling thread").emit(); } } }, 0, delay.getMillis(), TimeUnit.MILLISECONDS); started = true; } } @LifecycleStop public void stop() { synchronized (lock) { if (!started) { return; } started = false; dataSources.set(new ConcurrentHashMap<String, DruidDataSource>()); future.cancel(false); future = null; exec.shutdownNow(); exec = null; } } @Override public boolean enableDatasource(final String ds) { try { VersionedIntervalTimeline<String, DataSegment> segmentTimeline = dbi .withHandle(new HandleCallback<VersionedIntervalTimeline<String, DataSegment>>() { @Override public VersionedIntervalTimeline<String, DataSegment> withHandle(Handle handle) throws Exception { return handle .createQuery( String.format("SELECT payload FROM %s WHERE dataSource = :dataSource", getSegmentsTable())) .bind("dataSource", ds).map(ByteArrayMapper.FIRST) .fold(new VersionedIntervalTimeline<String, DataSegment>(Ordering.natural()), new Folder3<VersionedIntervalTimeline<String, DataSegment>, byte[]>() { @Override public VersionedIntervalTimeline<String, DataSegment> fold( VersionedIntervalTimeline<String, DataSegment> timeline, byte[] payload, FoldController foldController, StatementContext statementContext) throws SQLException { try { DataSegment segment = jsonMapper.readValue(payload, DataSegment.class); timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); return timeline; } catch (Exception e) { throw new SQLException(e.toString()); } } }); } }); final List<DataSegment> segments = Lists.newArrayList(); for (TimelineObjectHolder<String, DataSegment> objectHolder : segmentTimeline .lookup(new Interval("0000-01-01/3000-01-01"))) { for (PartitionChunk<DataSegment> partitionChunk : objectHolder.getObject()) { segments.add(partitionChunk.getObject()); } } if (segments.isEmpty()) { log.warn("No segments found in the database!"); return false; } dbi.withHandle(new HandleCallback<Void>() { @Override public Void withHandle(Handle handle) throws Exception { Batch batch = handle.createBatch(); for (DataSegment segment : segments) { batch.add(String.format("UPDATE %s SET used=true WHERE id = '%s'", getSegmentsTable(), segment.getIdentifier())); } batch.execute(); return null; } }); } catch (Exception e) { log.error(e, "Exception enabling datasource %s", ds); return false; } return true; } @Override public boolean enableSegment(final String segmentId) { try { dbi.withHandle(new HandleCallback<Void>() { @Override public Void withHandle(Handle handle) throws Exception { handle.createStatement( String.format("UPDATE %s SET used=true WHERE id = :id", getSegmentsTable())) .bind("id", segmentId).execute(); return null; } }); } catch (Exception e) { log.error(e, "Exception enabling segment %s", segmentId); return false; } return true; } @Override public boolean removeDatasource(final String ds) { try { ConcurrentHashMap<String, DruidDataSource> dataSourceMap = dataSources.get(); if (!dataSourceMap.containsKey(ds)) { log.warn("Cannot delete datasource %s, does not exist", ds); return false; } dbi.withHandle(new HandleCallback<Void>() { @Override public Void withHandle(Handle handle) throws Exception { handle.createStatement(String.format("UPDATE %s SET used=false WHERE dataSource = :dataSource", getSegmentsTable())).bind("dataSource", ds).execute(); return null; } }); dataSourceMap.remove(ds); } catch (Exception e) { log.error(e, "Error removing datasource %s", ds); return false; } return true; } @Override public boolean removeSegment(String ds, final String segmentID) { try { dbi.withHandle(new HandleCallback<Void>() { @Override public Void withHandle(Handle handle) throws Exception { handle.createStatement( String.format("UPDATE %s SET used=false WHERE id = :segmentID", getSegmentsTable())) .bind("segmentID", segmentID).execute(); return null; } }); ConcurrentHashMap<String, DruidDataSource> dataSourceMap = dataSources.get(); if (!dataSourceMap.containsKey(ds)) { log.warn("Cannot find datasource %s", ds); return false; } DruidDataSource dataSource = dataSourceMap.get(ds); dataSource.removePartition(segmentID); if (dataSource.isEmpty()) { dataSourceMap.remove(ds); } } catch (Exception e) { log.error(e, e.toString()); return false; } return true; } @Override public boolean isStarted() { return started; } @Override public DruidDataSource getInventoryValue(String key) { return dataSources.get().get(key); } @Override public Collection<DruidDataSource> getInventory() { return dataSources.get().values(); } @Override public Collection<String> getAllDatasourceNames() { synchronized (lock) { return dbi.withHandle(new HandleCallback<List<String>>() { @Override public List<String> withHandle(Handle handle) throws Exception { return handle .createQuery(String.format("SELECT DISTINCT(datasource) FROM %s", getSegmentsTable())) .fold(Lists.<String>newArrayList(), new Folder3<ArrayList<String>, Map<String, Object>>() { @Override public ArrayList<String> fold(ArrayList<String> druidDataSources, Map<String, Object> stringObjectMap, FoldController foldController, StatementContext statementContext) throws SQLException { druidDataSources.add(MapUtils.getString(stringObjectMap, "datasource")); return druidDataSources; } }); } }); } } @Override public void poll() { try { if (!started) { return; } ConcurrentHashMap<String, DruidDataSource> newDataSources = new ConcurrentHashMap<String, DruidDataSource>(); log.debug("Starting polling of segment table"); List<DataSegment> segments = dbi.withHandle(new HandleCallback<List<DataSegment>>() { @Override public List<DataSegment> withHandle(Handle handle) throws Exception { return handle .createQuery( String.format("SELECT payload FROM %s WHERE used=true", getSegmentsTable())) .map(new ResultSetMapper<DataSegment>() { @Override public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLException { try { return jsonMapper.readValue(r.getBytes("payload"), DataSegment.class); } catch (IOException e) { throw new SQLException(e); } } }).list(); } }); if (segments == null || segments.isEmpty()) { log.warn("No segments found in the database!"); return; } log.info("Polled and found %,d segments in the database", segments.size()); for (final DataSegment segment : segments) { String datasourceName = segment.getDataSource(); DruidDataSource dataSource = newDataSources.get(datasourceName); if (dataSource == null) { dataSource = new DruidDataSource(datasourceName, ImmutableMap.of("created", new DateTime().toString())); Object shouldBeNull = newDataSources.put(datasourceName, dataSource); if (shouldBeNull != null) { log.warn("Just put key[%s] into dataSources and what was there wasn't null!? It was[%s]", datasourceName, shouldBeNull); } } if (!dataSource.getSegments().contains(segment)) { dataSource.addSegment(segment.getIdentifier(), segment); } } synchronized (lock) { if (started) { dataSources.set(newDataSources); } } } catch (Exception e) { log.makeAlert(e, "Problem polling DB.").emit(); } } private String getSegmentsTable() { return dbTables.get().getSegmentsTable(); } }