001/* 002 * Copyright (c) 2016-2017 Chris K Wensel <chris@wensel.net>. All Rights Reserved. 003 * Copyright (c) 2007-2017 Xplenty, Inc. All Rights Reserved. 004 * 005 * Project and contact information: http://www.cascading.org/ 006 * 007 * This file is part of the Cascading project. 008 * 009 * Licensed under the Apache License, Version 2.0 (the "License"); 010 * you may not use this file except in compliance with the License. 011 * You may obtain a copy of the License at 012 * 013 * http://www.apache.org/licenses/LICENSE-2.0 014 * 015 * Unless required by applicable law or agreed to in writing, software 016 * distributed under the License is distributed on an "AS IS" BASIS, 017 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 018 * See the License for the specific language governing permissions and 019 * limitations under the License. 020 */ 021 022package cascading.flow.planner.rule; 023 024import java.util.Arrays; 025import java.util.Collection; 026import java.util.Collections; 027import java.util.Comparator; 028import java.util.LinkedList; 029 030/** 031 * Class RuleRegistrySet manages the set of {@link cascading.flow.planner.rule.RuleRegistry} instances that should be 032 * applied via the {@link cascading.flow.planner.FlowPlanner} to the given assembly. 033 * <p> 034 * The current RuleRegistrySet can be reached via {@link cascading.flow.FlowConnector#getRuleRegistrySet()}, or set with 035 * the appropriate {@link cascading.flow.FlowConnector} constructor sub-class. 036 * <p> 037 * RuleRegistrySet configuration is mutable to address different situations. 038 * <p> 039 * During planner execution, all RuleRegistry instances are applied simultaneously within individual threads. 040 * <p> 041 * If the planner duration exceeds {@link #getPlannerTimeoutSec()} (with default 042 * {@link cascading.flow.planner.rule.RuleSetExec#DEFAULT_TIMEOUT}) any incomplete planner executions 043 * will be cancelled. 044 * <p> 045 * If no planner executions complete successfully within the timeout period, an exception will be thrown. 046 * <p> 047 * If there are multiple successful completions, the default cost comparator, 048 * {@link cascading.flow.planner.rule.RuleSetExec#DEFAULT_PLAN_COMPARATOR}, will be applied to find the lower 049 * cost plan. Use {@link #setPlanComparator(java.util.Comparator)} to override. 050 * <p> 051 * If all plans have equivalent costs, the plan corresponding to the first most RuleRegistry, as given to the 052 * RuleRegistrySet, will be selected. 053 * <p> 054 * If {@link #setSelect(cascading.flow.planner.rule.RuleRegistrySet.Select)} is set to 055 * {@link cascading.flow.planner.rule.RuleRegistrySet.Select#FIRST}, the first RuleRegistry to complete will be used 056 * regardless of cost ordering provided by the plan Comparator. All remaining running planner executions will be 057 * cancelled. 058 * <p> 059 * If {@link #isIgnoreFailed()} is {@code false}, if any planner execution is times out or fails, an exception will be 060 * thrown. 061 */ 062public class RuleRegistrySet 063 { 064 public enum Select 065 { 066 FIRST, 067 COMPARED 068 } 069 070 int plannerTimeoutSec = RuleSetExec.DEFAULT_TIMEOUT; 071 boolean ignoreFailed = true; 072 Select select = Select.COMPARED; 073 Comparator<RuleResult> planComparator = RuleSetExec.DEFAULT_PLAN_COMPARATOR; 074 075 LinkedList<RuleRegistry> ruleRegistries = new LinkedList<>(); // preserve order, no duplicates 076 077 /** 078 * Instantiates a new Rule Registry Set. 079 * 080 * @param ruleRegistries the rule registries 081 */ 082 public RuleRegistrySet( RuleRegistry... ruleRegistries ) 083 { 084 this( Arrays.asList( ruleRegistries ) ); 085 } 086 087 /** 088 * Instantiates a new Rule Registry Set. 089 * 090 * @param ruleRegistries the rule registries 091 */ 092 public RuleRegistrySet( Collection<RuleRegistry> ruleRegistries ) 093 { 094 this.ruleRegistries.addAll( ruleRegistries ); 095 096 for( RuleRegistry ruleRegistry : ruleRegistries ) 097 { 098 if( Collections.frequency( this.ruleRegistries, ruleRegistry ) > 1 ) 099 throw new IllegalArgumentException( "may not include duplicate registries" ); 100 } 101 } 102 103 /** 104 * Gets planner timeout, in seconds. 105 * 106 * @return the planner timeout sec 107 */ 108 public int getPlannerTimeoutSec() 109 { 110 return plannerTimeoutSec; 111 } 112 113 /** 114 * Sets planner timeout, in seconds. 115 * <p> 116 * Extremely large assemblies may take longer than the default timeout. This allows for increasing 117 * that timeout period when necessary. 118 * 119 * @param plannerTimeoutSec the planner timeout sec 120 */ 121 public void setPlannerTimeoutSec( int plannerTimeoutSec ) 122 { 123 this.plannerTimeoutSec = plannerTimeoutSec; 124 } 125 126 /** 127 * When true, failures during planning will be ignored unless no registries are successful. When false, 128 * any failure will cause the planner to fail. 129 * 130 * @return type boolean 131 */ 132 public boolean isIgnoreFailed() 133 { 134 return ignoreFailed; 135 } 136 137 /** 138 * Sets ignore failed setting. The default is {@code true}. 139 * <p> 140 * When {@code true}, failures during planning will be ignored unless no registries are successful. When {@code false}, 141 * any failure will cause the planner to fail. 142 * 143 * @param ignoreFailed {@code true} if any failures should be ignored, the default 144 */ 145 public void setIgnoreFailed( boolean ignoreFailed ) 146 { 147 this.ignoreFailed = ignoreFailed; 148 } 149 150 /** 151 * Returns the result selection type. 152 * 153 * @return the selection type 154 */ 155 public Select getSelect() 156 { 157 return select; 158 } 159 160 /** 161 * Sets the result selection type. 162 * 163 * @param select the selection type 164 */ 165 public void setSelect( Select select ) 166 { 167 if( select == null ) 168 throw new IllegalArgumentException( "select may not be null" ); 169 170 this.select = select; 171 } 172 173 /** 174 * Returns planner comparator. 175 * 176 * @return the planner result comparator 177 */ 178 public Comparator<RuleResult> getPlanComparator() 179 { 180 return planComparator; 181 } 182 183 /** 184 * Sets planner result comparator, used only if {@link #getSelect()} is {@link Select#COMPARED}. 185 * 186 * @param planComparator the plan comparator 187 */ 188 public void setPlanComparator( Comparator<RuleResult> planComparator ) 189 { 190 if( planComparator == null ) 191 throw new IllegalArgumentException( "planComparator may not be null" ); 192 193 this.planComparator = planComparator; 194 } 195 196 public int size() 197 { 198 return ruleRegistries.size(); 199 } 200 201 public RuleRegistry findRegistryWith( String ruleName ) 202 { 203 for( RuleRegistry ruleRegistry : ruleRegistries ) 204 { 205 if( ruleRegistry.hasRule( ruleName ) ) 206 return ruleRegistry; 207 } 208 209 return null; 210 } 211 212 protected int indexOf( RuleRegistry ruleRegistry ) 213 { 214 return ruleRegistries.indexOf( ruleRegistry ); 215 } 216 }