@@ -53,54 +53,67 @@ public DataSourceReader createReader(DataSourceOptions options) {
5353 * and how it obtains the reader factories to be used by the executors to create readers.
5454 * Notice that one factory is created for each partition.
5555 */
56- class Reader implements DataSourceReader , SupportsReportPartitioning {
56+ static class Reader implements DataSourceReader , SupportsReportPartitioning {
57+
58+ static Logger log = Logger .getLogger (Reader .class .getName ());
5759
5860 public Reader (String host , int port , String table , int partitions ) {
5961 _host = host ;
6062 _port = port ;
6163 _table = table ;
62- _partitions = partitions ;
64+ _requestedPartitions = partitions ;
6365 }
6466
65- private StructType _schema ;
6667 private String _host ;
6768 private int _port ;
6869 private String _table ;
69- private int _partitions ;
70+ private int _requestedPartitions ;
7071
71- @ Override
72- public StructType readSchema () {
73- if (_schema == null ) {
72+ //
73+ // dynamic properties inferred from database
74+ //
75+
76+ private boolean _initialized = false ;
77+ private StructType _schema ;
78+ private String _clusteredColumn ;
79+ private List <Split > _splits ;
80+
81+
82+ private void initialize () {
83+ if (!_initialized ) {
84+ log .info ("initializing" );
7485 DBClientWrapper db = new DBClientWrapper (_host , _port );
7586 db .connect ();
7687 try {
7788 _schema = db .getSchema (_table );
89+ _clusteredColumn = db .getClusteredIndexColumn (_table );
90+ if (_requestedPartitions == 0 )
91+ _splits = db .getSplits (_table );
92+ else
93+ _splits = db .getSplits (_table , _requestedPartitions );
7894 } catch (UnknownTableException ute ) {
7995 throw new RuntimeException (ute );
8096 } finally {
8197 db .disconnect ();
8298 }
99+ _initialized = true ;
100+ log .info ("initialized" );
83101 }
102+ }
103+
104+ @ Override
105+ public StructType readSchema () {
106+ log .info ("schema requested for table [" + _table + "]" );
107+ initialize ();
84108 return _schema ;
85109 }
86110
87111 @ Override
88112 public List <DataReaderFactory <Row >> createDataReaderFactories () {
89- List <Split > splits = null ;
90- DBClientWrapper db = new DBClientWrapper (_host , _port );
91- db .connect ();
92- try {
93- if (_partitions == 0 )
94- splits = db .getSplits (_table );
95- else
96- splits = db .getSplits (_table , _partitions );
97- } catch (UnknownTableException ute ) {
98- throw new RuntimeException (ute );
99- } finally {
100- db .disconnect ();
101- }
113+ log .info ("reader factories requested for table [" + _table + "]" );
114+ initialize ();
102115 List <DataReaderFactory <Row >> factories = new ArrayList <>();
103- for (Split split : splits ) {
116+ for (Split split : _splits ) {
104117 DataReaderFactory <Row > factory =
105118 new SplitDataReaderFactory (_host , _port , _table , readSchema (), split );
106119 factories .add (factory );
@@ -110,37 +123,24 @@ public List<DataReaderFactory<Row>> createDataReaderFactories() {
110123
111124 @ Override
112125 public Partitioning outputPartitioning () {
113- return new TrivialPartitioning ();
114- }
115- }
116-
117- static class TrivialPartitioning implements Partitioning {
118-
119- static Logger log = Logger .getLogger (TrivialPartitioning .class .getName ());
120-
121- @ Override
122- public int numPartitions () {
123- log .info ("asked for numPartitions" );
124- return 8 ;
125- }
126-
127- @ Override
128- public boolean satisfy (Distribution distribution ) {
129- log .info ("asked to satisfy" );
130- // can't satisfy any Distribution
131- return false ;
126+ log .info ("output partitioning requested for table [" + _table + "]" );
127+ return new SingleClusteredColumnPartitioning (
128+ _clusteredColumn , _splits .size ());
132129 }
133130 }
134131
135132 static class SingleClusteredColumnPartitioning implements Partitioning {
136133
134+ static Logger log = Logger .getLogger (SingleClusteredColumnPartitioning .class .getName ());
135+
137136 public SingleClusteredColumnPartitioning (String columnName , int partitions ) {
138137 _columnName = columnName ;
139138 _partitions = partitions ;
140139 }
141140
142141 @ Override
143142 public int numPartitions () {
143+ log .info ("asked for numPartitions" );
144144 return _partitions ;
145145 }
146146
@@ -150,11 +150,31 @@ public boolean satisfy(Distribution distribution) {
150150 // Since Spark may add other Distribution policies in the future, we can't assume
151151 // it's always a ClusteredDistribution
152152 //
153+
153154 if (distribution instanceof ClusteredDistribution ) {
155+
154156 String [] clusteredCols = ((ClusteredDistribution ) distribution ).clusteredColumns ;
155- return Arrays .asList (clusteredCols ).contains (_columnName );
157+ StringBuilder logEntryBuilder = new StringBuilder ();
158+ logEntryBuilder .append ("asked to satisfy ClusteredDistribution on columns " );
159+ if (clusteredCols .length > 0 ) {
160+ for (String col : clusteredCols ) {
161+ logEntryBuilder .append ("[" );
162+ logEntryBuilder .append (col );
163+ logEntryBuilder .append ("] " );
164+ }
165+ }
166+ log .info (logEntryBuilder .toString ());
167+ if (_columnName == null ) {
168+ log .info ("no cluster column so does not satisfy" );
169+ return false ;
170+ } else {
171+ boolean satisfies = Arrays .asList (clusteredCols ).contains (_columnName );
172+ log .info ("based on cluster column: " + satisfies );
173+ return satisfies ;
174+ }
156175 }
157-
176+ log .info ("asked to satisfy unknown distribution of type [" +
177+ distribution .getClass ().getCanonicalName () + "]" );
158178 return false ;
159179 }
160180
@@ -207,6 +227,8 @@ public void close() throws IOException {
207227 */
208228 static class SplitDataReaderFactory implements DataReaderFactory <Row > {
209229
230+ static Logger log = Logger .getLogger (SplitDataReaderFactory .class .getName ());
231+
210232 public SplitDataReaderFactory (String host , int port ,
211233 String table , StructType schema ,
212234 Split split ) {
0 commit comments